From 2f846d6a6fcf13f9c78b8c35b37f682ad5083dcb Mon Sep 17 00:00:00 2001 From: Matt Butrovich Date: Mon, 20 Apr 2026 17:20:36 -0400 Subject: [PATCH 01/21] Implement non-AQE DPP for CometNativeScanExec. --- .../scala/org/apache/comet/CometConf.scala | 3 +- .../apache/comet/rules/CometScanRule.scala | 11 ++- .../serde/operator/CometNativeScan.scala | 32 ++++--- .../spark/sql/comet/CometNativeScanExec.scala | 30 +++++++ .../apache/comet/exec/CometExecSuite.scala | 84 ++++++++++++++++++- 5 files changed, 146 insertions(+), 14 deletions(-) diff --git a/common/src/main/scala/org/apache/comet/CometConf.scala b/common/src/main/scala/org/apache/comet/CometConf.scala index baf0c7bd77..2395e63081 100644 --- a/common/src/main/scala/org/apache/comet/CometConf.scala +++ b/common/src/main/scala/org/apache/comet/CometConf.scala @@ -557,7 +557,8 @@ object CometConf extends ShimCometConf { val COMET_DPP_FALLBACK_ENABLED: ConfigEntry[Boolean] = conf("spark.comet.dppFallback.enabled") .category(CATEGORY_EXEC) - .doc("Whether to fall back to Spark for queries that use DPP.") + .doc("Whether to fall back to Spark for queries that use AQE Dynamic Partition Pruning " + + "(SubqueryAdaptiveBroadcastExec). Non-AQE DPP is always supported natively.") .booleanConf .createWithDefault(true) diff --git a/spark/src/main/scala/org/apache/comet/rules/CometScanRule.scala b/spark/src/main/scala/org/apache/comet/rules/CometScanRule.scala index dc0e7099c9..6ae453fa68 100644 --- a/spark/src/main/scala/org/apache/comet/rules/CometScanRule.scala +++ b/spark/src/main/scala/org/apache/comet/rules/CometScanRule.scala @@ -140,8 +140,8 @@ case class CometScanRule(session: SparkSession) private def transformV1Scan(plan: SparkPlan, scanExec: FileSourceScanExec): SparkPlan = { if (COMET_DPP_FALLBACK_ENABLED.get() && - scanExec.partitionFilters.exists(isDynamicPruningFilter)) { - return withInfo(scanExec, "Dynamic Partition Pruning is not supported") + scanExec.partitionFilters.exists(isAqeDynamicPruningFilter)) { + return withInfo(scanExec, "AQE Dynamic Partition Pruning is not supported") } scanExec.relation match { @@ -647,6 +647,13 @@ case class CometScanRule(session: SparkSession) private def isDynamicPruningFilter(e: Expression): Boolean = e.exists(_.isInstanceOf[PlanExpression[_]]) + /** Detects AQE DPP (SubqueryAdaptiveBroadcastExec), as opposed to non-AQE DPP. */ + private def isAqeDynamicPruningFilter(e: Expression): Boolean = + e.exists { + case sub: InSubqueryExec => sub.plan.isInstanceOf[SubqueryAdaptiveBroadcastExec] + case _ => false + } + private def isSchemaSupported( scanExec: FileSourceScanExec, scanImpl: String, diff --git a/spark/src/main/scala/org/apache/comet/serde/operator/CometNativeScan.scala b/spark/src/main/scala/org/apache/comet/serde/operator/CometNativeScan.scala index d5d075760f..6d7467be6f 100644 --- a/spark/src/main/scala/org/apache/comet/serde/operator/CometNativeScan.scala +++ b/spark/src/main/scala/org/apache/comet/serde/operator/CometNativeScan.scala @@ -26,7 +26,7 @@ import org.apache.spark.internal.Logging import org.apache.spark.sql.catalyst.expressions.{Expression, Literal, PlanExpression} import org.apache.spark.sql.catalyst.util.ResolveDefaultColumns.getExistenceDefaultValues import org.apache.spark.sql.comet.{CometNativeExec, CometNativeScanExec, CometScanExec} -import org.apache.spark.sql.execution.FileSourceScanExec +import org.apache.spark.sql.execution.{FileSourceScanExec, InSubqueryExec, SubqueryAdaptiveBroadcastExec} import org.apache.spark.sql.execution.datasources.PartitionedFile import org.apache.spark.sql.internal.SQLConf @@ -57,9 +57,11 @@ object CometNativeScan extends CometOperatorSerde[CometScanExec] with Logging { withInfo(scanExec, s"Full native scan disabled because ${COMET_EXEC_ENABLED.key} disabled") } - // Native DataFusion doesn't support subqueries/dynamic pruning - if (scanExec.partitionFilters.exists(isDynamicPruningFilter)) { - withInfo(scanExec, "Native DataFusion scan does not support subqueries/dynamic pruning") + // Native DataFusion doesn't support AQE DPP (SubqueryAdaptiveBroadcastExec). + // Non-AQE DPP (SubqueryBroadcastExec/SubqueryExec) is supported through the lazy + // partition serialization path in CometNativeScanExec. + if (scanExec.partitionFilters.exists(isAqeDynamicPruningFilter)) { + withInfo(scanExec, "Native DataFusion scan does not support AQE dynamic pruning") } if (SQLConf.get.ignoreCorruptFiles || @@ -84,6 +86,13 @@ object CometNativeScan extends CometOperatorSerde[CometScanExec] with Logging { private def isDynamicPruningFilter(e: Expression): Boolean = e.exists(_.isInstanceOf[PlanExpression[_]]) + /** Detects AQE DPP (SubqueryAdaptiveBroadcastExec), as opposed to non-AQE DPP. */ + private def isAqeDynamicPruningFilter(e: Expression): Boolean = + e.exists { + case sub: InSubqueryExec => sub.plan.isInstanceOf[SubqueryAdaptiveBroadcastExec] + case _ => false + } + override def enabledConfig: Option[ConfigEntry[Boolean]] = None override def getSupportLevel(operator: CometScanExec): SupportLevel = { @@ -144,10 +153,13 @@ object CometNativeScan extends CometOperatorSerde[CometScanExec] with Logging { commonBuilder.addAllDefaultValuesIndexes(indexes.toIterable.asJava) } - // Extract object store options from first file (S3 configs apply to all files in scan) - var firstPartition: Option[PartitionedFile] = None - val filePartitions = scan.getFilePartitions() - firstPartition = filePartitions.flatMap(_.files.headOption).headOption + // Extract object store options from first file (S3 configs apply to all files in scan). + // Use selectedPartitions (static) instead of getFilePartitions() because at planning time + // DPP subqueries haven't been resolved yet. Object store options don't depend on DPP. + val firstFileUri = scan.selectedPartitions + .flatMap(_.files.headOption) + .headOption + .map(_.getPath.toUri) val partitionSchema = schema2Proto(scan.relation.partitionSchema.fields) val requiredSchema = schema2Proto(scan.requiredSchema.fields) @@ -181,9 +193,9 @@ object CometNativeScan extends CometOperatorSerde[CometScanExec] with Logging { commonBuilder.setEncryptionEnabled(CometParquetUtils.encryptionEnabled(hadoopConf)) - firstPartition.foreach { partitionFile => + firstFileUri.foreach { uri => val objectStoreOptions = - NativeConfig.extractObjectStoreOptions(hadoopConf, partitionFile.pathUri) + NativeConfig.extractObjectStoreOptions(hadoopConf, uri) objectStoreOptions.foreach { case (key, value) => commonBuilder.putObjectStoreOptions(key, value) } diff --git a/spark/src/main/scala/org/apache/spark/sql/comet/CometNativeScanExec.scala b/spark/src/main/scala/org/apache/spark/sql/comet/CometNativeScanExec.scala index ae2d873ef7..9b134873fc 100644 --- a/spark/src/main/scala/org/apache/spark/sql/comet/CometNativeScanExec.scala +++ b/spark/src/main/scala/org/apache/spark/sql/comet/CometNativeScanExec.scala @@ -76,6 +76,27 @@ case class CometNativeScanExec( override lazy val metadata: Map[String, String] = originalPlan.metadata + /** + * Prepare DPP subquery plans before execution. + * + * For non-AQE DPP, partitionFilters contains DynamicPruningExpression(InSubqueryExec(...)) + * inserted by PlanDynamicPruningFilters (which runs before Comet rules). We call + * e.plan.prepare() here so that the subquery plans are set up before execution begins. + * + * Note: doPrepare() alone is NOT sufficient for DPP resolution. serializedPartitionData can be + * triggered from findAllPlanData (via commonData) on a BroadcastExchangeExec thread, outside + * the normal prepare() -> executeSubqueries() flow. The actual DPP resolution (updateResult) + * happens in serializedPartitionData below. + */ + override protected def doPrepare(): Unit = { + partitionFilters.foreach { + case DynamicPruningExpression(e: InSubqueryExec) => + e.plan.prepare() + case _ => + } + super.doPrepare() + } + override val nodeName: String = s"CometNativeScan $relation ${tableIdentifier.map(_.unquotedString).getOrElse("")}" @@ -141,6 +162,15 @@ case class CometNativeScanExec( * partition's files (lazily, as tasks are scheduled). */ @transient private lazy val serializedPartitionData: (Array[Byte], Array[Array[Byte]]) = { + // Ensure DPP subqueries are resolved before accessing file partitions. + // serializedPartitionData can be triggered from findAllPlanData (via commonData) on a + // different execution path than the standard prepare() -> executeSubqueries() flow + // (e.g., from a BroadcastExchangeExec thread). We must resolve DPP here explicitly. + partitionFilters.foreach { + case DynamicPruningExpression(e: InSubqueryExec) if e.values().isEmpty => + e.updateResult() + case _ => + } // Extract common data from nativeOp val commonBytes = nativeOp.getNativeScan.getCommon.toByteArray diff --git a/spark/src/test/scala/org/apache/comet/exec/CometExecSuite.scala b/spark/src/test/scala/org/apache/comet/exec/CometExecSuite.scala index 22983119bb..fab3862cfe 100644 --- a/spark/src/test/scala/org/apache/comet/exec/CometExecSuite.scala +++ b/spark/src/test/scala/org/apache/comet/exec/CometExecSuite.scala @@ -138,7 +138,7 @@ class CometExecSuite extends CometTestBase { "select * from dpp_fact join dpp_dim on fact_date = dim_date where dim_id > 7") val (_, cometPlan) = checkSparkAnswer(df) val infos = new ExtendedExplainInfo().generateExtendedInfo(cometPlan) - assert(infos.contains("Dynamic Partition Pruning is not supported")) + assert(infos.contains("AQE Dynamic Partition Pruning is not supported")) } } } @@ -182,6 +182,88 @@ class CometExecSuite extends CometTestBase { } } + test("non-AQE DPP with BHJ works with CometNativeScanExec") { + withTempDir { path => + val factPath = s"${path.getAbsolutePath}/fact.parquet" + val dimPath = s"${path.getAbsolutePath}/dim.parquet" + withSQLConf(CometConf.COMET_EXEC_ENABLED.key -> "false") { + val one_day = 24 * 60 * 60000 + val fact = Range(0, 100) + .map(i => (i, new java.sql.Date(System.currentTimeMillis() + (i % 10) * one_day))) + .toDF("fact_id", "fact_date") + fact.write.partitionBy("fact_date").parquet(factPath) + val dim = Range(0, 10) + .map(i => (i, new java.sql.Date(System.currentTimeMillis() + i * one_day))) + .toDF("dim_id", "dim_date") + dim.write.parquet(dimPath) + } + + // AQE off ensures PlanDynamicPruningFilters (non-AQE) creates the DPP filters + // with SubqueryBroadcastExec, not SubqueryAdaptiveBroadcastExec + withSQLConf( + SQLConf.USE_V1_SOURCE_LIST.key -> "parquet", + SQLConf.ADAPTIVE_EXECUTION_ENABLED.key -> "false", + CometConf.COMET_DPP_FALLBACK_ENABLED.key -> "true") { + spark.read.parquet(factPath).createOrReplaceTempView("dpp_fact_bhj") + spark.read.parquet(dimPath).createOrReplaceTempView("dpp_dim_bhj") + val df = spark.sql( + "select * from dpp_fact_bhj join dpp_dim_bhj on fact_date = dim_date where dim_id > 7") + val (_, cometPlan) = checkSparkAnswer(df) + + assert( + cometPlan.toString().contains("CometNativeScan"), + s"Expected CometNativeScan in plan:\n${cometPlan.toString()}") + + val infos = new ExtendedExplainInfo().generateExtendedInfo(cometPlan) + assert( + !infos.contains("AQE Dynamic Partition Pruning is not supported"), + s"Should not fall back for non-AQE DPP:\n$infos") + } + } + } + + test("non-AQE DPP with SMJ works with CometNativeScanExec") { + withTempDir { path => + val factPath = s"${path.getAbsolutePath}/fact.parquet" + val dimPath = s"${path.getAbsolutePath}/dim.parquet" + withSQLConf(CometConf.COMET_EXEC_ENABLED.key -> "false") { + val one_day = 24 * 60 * 60000 + val fact = Range(0, 100) + .map(i => (i, new java.sql.Date(System.currentTimeMillis() + (i % 10) * one_day))) + .toDF("fact_id", "fact_date") + fact.write.partitionBy("fact_date").parquet(factPath) + val dim = Range(0, 10) + .map(i => (i, new java.sql.Date(System.currentTimeMillis() + i * one_day))) + .toDF("dim_id", "dim_date") + dim.write.parquet(dimPath) + } + + // AQE off + broadcast disabled → SMJ is used. PlanDynamicPruningFilters can't reuse + // broadcast, so DPP uses SubqueryExec (aggregate) or Literal.TrueLiteral (if + // onlyInBroadcast). Either way, non-AQE DPP should work natively. + withSQLConf( + SQLConf.USE_V1_SOURCE_LIST.key -> "parquet", + SQLConf.ADAPTIVE_EXECUTION_ENABLED.key -> "false", + SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> "-1", + CometConf.COMET_DPP_FALLBACK_ENABLED.key -> "true") { + spark.read.parquet(factPath).createOrReplaceTempView("dpp_fact_smj") + spark.read.parquet(dimPath).createOrReplaceTempView("dpp_dim_smj") + val df = spark.sql( + "select * from dpp_fact_smj join dpp_dim_smj on fact_date = dim_date where dim_id > 7") + val (_, cometPlan) = checkSparkAnswer(df) + + assert( + cometPlan.toString().contains("CometNativeScan"), + s"Expected CometNativeScan in plan:\n${cometPlan.toString()}") + + val infos = new ExtendedExplainInfo().generateExtendedInfo(cometPlan) + assert( + !infos.contains("AQE Dynamic Partition Pruning is not supported"), + s"Should not fall back for non-AQE DPP:\n$infos") + } + } + } + test("ShuffleQueryStageExec could be direct child node of CometBroadcastExchangeExec") { withSQLConf(CometConf.COMET_SHUFFLE_MODE.key -> "jvm") { val table = "src" From dcecb78ba240845ab928512d5cbcf23f606b8c4e Mon Sep 17 00:00:00 2001 From: Matt Butrovich Date: Mon, 20 Apr 2026 17:44:22 -0400 Subject: [PATCH 02/21] Fix tests. --- .../apache/comet/exec/CometExecSuite.scala | 22 +++++++++++-------- 1 file changed, 13 insertions(+), 9 deletions(-) diff --git a/spark/src/test/scala/org/apache/comet/exec/CometExecSuite.scala b/spark/src/test/scala/org/apache/comet/exec/CometExecSuite.scala index fab3862cfe..868f8b5244 100644 --- a/spark/src/test/scala/org/apache/comet/exec/CometExecSuite.scala +++ b/spark/src/test/scala/org/apache/comet/exec/CometExecSuite.scala @@ -31,7 +31,7 @@ import org.apache.hadoop.fs.Path import org.apache.spark.sql._ import org.apache.spark.sql.catalyst.{FunctionIdentifier, TableIdentifier} import org.apache.spark.sql.catalyst.catalog.{BucketSpec, CatalogStatistics, CatalogTable} -import org.apache.spark.sql.catalyst.expressions.{Expression, ExpressionInfo, Hex} +import org.apache.spark.sql.catalyst.expressions.{DynamicPruningExpression, Expression, ExpressionInfo, Hex} import org.apache.spark.sql.catalyst.expressions.aggregate.{AggregateMode, BloomFilterAggregate} import org.apache.spark.sql.comet._ import org.apache.spark.sql.comet.execution.shuffle.{CometColumnarShuffle, CometShuffleExchangeExec} @@ -208,11 +208,16 @@ class CometExecSuite extends CometTestBase { spark.read.parquet(dimPath).createOrReplaceTempView("dpp_dim_bhj") val df = spark.sql( "select * from dpp_fact_bhj join dpp_dim_bhj on fact_date = dim_date where dim_id > 7") - val (_, cometPlan) = checkSparkAnswer(df) + val (_, cometPlan) = checkSparkAnswerAndOperator(df) + val nativeScans = cometPlan.collect { case s: CometNativeScanExec => s } + assert(nativeScans.nonEmpty, "Expected CometNativeScanExec in plan") + + val dppScans = + nativeScans.filter(_.partitionFilters.exists(_.isInstanceOf[DynamicPruningExpression])) assert( - cometPlan.toString().contains("CometNativeScan"), - s"Expected CometNativeScan in plan:\n${cometPlan.toString()}") + dppScans.nonEmpty, + "Expected at least one CometNativeScanExec with DynamicPruningExpression") val infos = new ExtendedExplainInfo().generateExtendedInfo(cometPlan) assert( @@ -238,7 +243,7 @@ class CometExecSuite extends CometTestBase { dim.write.parquet(dimPath) } - // AQE off + broadcast disabled → SMJ is used. PlanDynamicPruningFilters can't reuse + // AQE off + broadcast disabled -> SMJ is used. PlanDynamicPruningFilters can't reuse // broadcast, so DPP uses SubqueryExec (aggregate) or Literal.TrueLiteral (if // onlyInBroadcast). Either way, non-AQE DPP should work natively. withSQLConf( @@ -250,11 +255,10 @@ class CometExecSuite extends CometTestBase { spark.read.parquet(dimPath).createOrReplaceTempView("dpp_dim_smj") val df = spark.sql( "select * from dpp_fact_smj join dpp_dim_smj on fact_date = dim_date where dim_id > 7") - val (_, cometPlan) = checkSparkAnswer(df) + val (_, cometPlan) = checkSparkAnswerAndOperator(df) - assert( - cometPlan.toString().contains("CometNativeScan"), - s"Expected CometNativeScan in plan:\n${cometPlan.toString()}") + val nativeScans = cometPlan.collect { case s: CometNativeScanExec => s } + assert(nativeScans.nonEmpty, "Expected CometNativeScanExec in plan") val infos = new ExtendedExplainInfo().generateExtendedInfo(cometPlan) assert( From cc583ecb397c503738557a4354890291d0c936b4 Mon Sep 17 00:00:00 2001 From: Matt Butrovich Date: Mon, 20 Apr 2026 18:04:54 -0400 Subject: [PATCH 03/21] Format. --- .../scala/org/apache/comet/serde/operator/CometNativeScan.scala | 1 - 1 file changed, 1 deletion(-) diff --git a/spark/src/main/scala/org/apache/comet/serde/operator/CometNativeScan.scala b/spark/src/main/scala/org/apache/comet/serde/operator/CometNativeScan.scala index 6d7467be6f..da7f24183b 100644 --- a/spark/src/main/scala/org/apache/comet/serde/operator/CometNativeScan.scala +++ b/spark/src/main/scala/org/apache/comet/serde/operator/CometNativeScan.scala @@ -27,7 +27,6 @@ import org.apache.spark.sql.catalyst.expressions.{Expression, Literal, PlanExpre import org.apache.spark.sql.catalyst.util.ResolveDefaultColumns.getExistenceDefaultValues import org.apache.spark.sql.comet.{CometNativeExec, CometNativeScanExec, CometScanExec} import org.apache.spark.sql.execution.{FileSourceScanExec, InSubqueryExec, SubqueryAdaptiveBroadcastExec} -import org.apache.spark.sql.execution.datasources.PartitionedFile import org.apache.spark.sql.internal.SQLConf import org.apache.comet.{CometConf, ConfigEntry} From c16498bd7c2be182e4117030f0ded0b4f0499318 Mon Sep 17 00:00:00 2001 From: Matt Butrovich Date: Mon, 20 Apr 2026 20:56:57 -0400 Subject: [PATCH 04/21] Update golden plans. --- .../spark/sql/comet/CometNativeScanExec.scala | 16 +- .../q1.native_datafusion/extended.txt | 111 +- .../q10.native_datafusion/extended.txt | 72 +- .../q11.native_datafusion/extended.txt | 203 +- .../q12.native_datafusion/extended.txt | 54 +- .../q13.native_datafusion/extended.txt | 88 +- .../q14a.native_datafusion/extended.txt | 1099 ++-- .../q14b.native_datafusion/extended.txt | 796 ++- .../q15.native_datafusion/extended.txt | 66 +- .../q17.native_datafusion/extended.txt | 135 +- .../q18.native_datafusion/extended.txt | 107 +- .../q20.native_datafusion/extended.txt | 54 +- .../q21.native_datafusion/extended.txt | 64 +- .../q22.native_datafusion/extended.txt | 68 +- .../q23a.native_datafusion/extended.txt | 203 +- .../q23b.native_datafusion/extended.txt | 207 +- .../q25.native_datafusion/extended.txt | 135 +- .../q26.native_datafusion/extended.txt | 81 +- .../q27.native_datafusion/extended.txt | 83 +- .../q29.native_datafusion/extended.txt | 140 +- .../q30.native_datafusion/extended.txt | 137 +- .../q31.native_datafusion/extended.txt | 282 +- .../q32.native_datafusion/extended.txt | 93 +- .../q33.native_datafusion/extended.txt | 217 +- .../q34.native_datafusion/extended.txt | 82 +- .../q35.native_datafusion/extended.txt | 72 +- .../q36.native_datafusion/extended.txt | 69 +- .../q37.native_datafusion/extended.txt | 64 +- .../q38.native_datafusion/extended.txt | 133 +- .../q39a.native_datafusion/extended.txt | 134 +- .../q39b.native_datafusion/extended.txt | 134 +- .../q4.native_datafusion/extended.txt | 299 +- .../q40.native_datafusion/extended.txt | 19 +- .../q45.native_datafusion/extended.txt | 66 +- .../q46.native_datafusion/extended.txt | 99 +- .../q47.native_datafusion/extended.txt | 179 +- .../q48.native_datafusion/extended.txt | 77 +- .../q49.native_datafusion/extended.txt | 154 +- .../q5.native_datafusion/extended.txt | 219 +- .../q50.native_datafusion/extended.txt | 77 +- .../q51.native_datafusion/extended.txt | 75 +- .../q53.native_datafusion/extended.txt | 65 +- .../q54.native_datafusion/extended.txt | 210 +- .../q56.native_datafusion/extended.txt | 223 +- .../q57.native_datafusion/extended.txt | 179 +- .../q58.native_datafusion/extended.txt | 263 +- .../q6.native_datafusion/extended.txt | 123 +- .../q60.native_datafusion/extended.txt | 223 +- .../q61.native_datafusion/extended.txt | 188 +- .../q63.native_datafusion/extended.txt | 65 +- .../q64.native_datafusion/extended.txt | 62 +- .../q65.native_datafusion/extended.txt | 115 +- .../q66.native_datafusion/extended.txt | 154 +- .../q67.native_datafusion/extended.txt | 72 +- .../q68.native_datafusion/extended.txt | 99 +- .../q69.native_datafusion/extended.txt | 72 +- .../q7.native_datafusion/extended.txt | 81 +- .../q70.native_datafusion/extended.txt | 77 +- .../q71.native_datafusion/extended.txt | 119 +- .../q72.native_datafusion/extended.txt | 126 +- .../q73.native_datafusion/extended.txt | 82 +- .../q74.native_datafusion/extended.txt | 201 +- .../q75.native_datafusion/extended.txt | 232 +- .../q77.native_datafusion/extended.txt | 257 +- .../q78.native_datafusion/extended.txt | 45 +- .../q79.native_datafusion/extended.txt | 75 +- .../q8.native_datafusion/extended.txt | 106 +- .../q80.native_datafusion/extended.txt | 49 +- .../q81.native_datafusion/extended.txt | 137 +- .../q82.native_datafusion/extended.txt | 64 +- .../q83.native_datafusion/extended.txt | 247 +- .../q85.native_datafusion/extended.txt | 118 +- .../q86.native_datafusion/extended.txt | 56 +- .../q87.native_datafusion/extended.txt | 133 +- .../q89.native_datafusion/extended.txt | 65 +- .../q91.native_datafusion/extended.txt | 104 +- .../q92.native_datafusion/extended.txt | 93 +- .../q97.native_datafusion/extended.txt | 61 +- .../q98.native_datafusion/extended.txt | 54 +- .../q1.native_datafusion/extended.txt | 111 +- .../q10.native_datafusion/extended.txt | 72 +- .../q11.native_datafusion/extended.txt | 203 +- .../q12.native_datafusion/extended.txt | 54 +- .../q13.native_datafusion/extended.txt | 88 +- .../q14a.native_datafusion/extended.txt | 1099 ++-- .../q14b.native_datafusion/extended.txt | 820 +-- .../q15.native_datafusion/extended.txt | 66 +- .../q17.native_datafusion/extended.txt | 135 +- .../q18.native_datafusion/extended.txt | 107 +- .../q20.native_datafusion/extended.txt | 54 +- .../q21.native_datafusion/extended.txt | 64 +- .../q22.native_datafusion/extended.txt | 68 +- .../q23a.native_datafusion/extended.txt | 203 +- .../q23b.native_datafusion/extended.txt | 207 +- .../q25.native_datafusion/extended.txt | 135 +- .../q26.native_datafusion/extended.txt | 81 +- .../q27.native_datafusion/extended.txt | 83 +- .../q29.native_datafusion/extended.txt | 140 +- .../q30.native_datafusion/extended.txt | 137 +- .../q31.native_datafusion/extended.txt | 282 +- .../q32.native_datafusion/extended.txt | 93 +- .../q33.native_datafusion/extended.txt | 217 +- .../q34.native_datafusion/extended.txt | 82 +- .../q35.native_datafusion/extended.txt | 72 +- .../q36.native_datafusion/extended.txt | 69 +- .../q37.native_datafusion/extended.txt | 64 +- .../q38.native_datafusion/extended.txt | 133 +- .../q39a.native_datafusion/extended.txt | 134 +- .../q39b.native_datafusion/extended.txt | 134 +- .../q4.native_datafusion/extended.txt | 299 +- .../q40.native_datafusion/extended.txt | 19 +- .../q45.native_datafusion/extended.txt | 66 +- .../q46.native_datafusion/extended.txt | 99 +- .../q47.native_datafusion/extended.txt | 179 +- .../q48.native_datafusion/extended.txt | 77 +- .../q49.native_datafusion/extended.txt | 154 +- .../q5.native_datafusion/extended.txt | 219 +- .../q50.native_datafusion/extended.txt | 77 +- .../q51.native_datafusion/extended.txt | 75 +- .../q53.native_datafusion/extended.txt | 65 +- .../q54.native_datafusion/extended.txt | 244 +- .../q56.native_datafusion/extended.txt | 223 +- .../q57.native_datafusion/extended.txt | 179 +- .../q58.native_datafusion/extended.txt | 285 +- .../q6.native_datafusion/extended.txt | 141 +- .../q60.native_datafusion/extended.txt | 223 +- .../q61.native_datafusion/extended.txt | 188 +- .../q63.native_datafusion/extended.txt | 65 +- .../q64.native_datafusion/extended.txt | 62 +- .../q65.native_datafusion/extended.txt | 115 +- .../q66.native_datafusion/extended.txt | 154 +- .../q67.native_datafusion/extended.txt | 72 +- .../q68.native_datafusion/extended.txt | 99 +- .../q69.native_datafusion/extended.txt | 72 +- .../q7.native_datafusion/extended.txt | 81 +- .../q70.native_datafusion/extended.txt | 77 +- .../q71.native_datafusion/extended.txt | 119 +- .../q72.native_datafusion/extended.txt | 126 +- .../q73.native_datafusion/extended.txt | 82 +- .../q74.native_datafusion/extended.txt | 201 +- .../q75.native_datafusion/extended.txt | 232 +- .../q77.native_datafusion/extended.txt | 257 +- .../q78.native_datafusion/extended.txt | 45 +- .../q79.native_datafusion/extended.txt | 75 +- .../q8.native_datafusion/extended.txt | 106 +- .../q80.native_datafusion/extended.txt | 49 +- .../q81.native_datafusion/extended.txt | 137 +- .../q82.native_datafusion/extended.txt | 64 +- .../q83.ansi.native_datafusion/extended.txt | 247 +- .../q85.native_datafusion/extended.txt | 118 +- .../q86.native_datafusion/extended.txt | 56 +- .../q87.native_datafusion/extended.txt | 133 +- .../q89.native_datafusion/extended.txt | 65 +- .../q91.native_datafusion/extended.txt | 104 +- .../q92.native_datafusion/extended.txt | 93 +- .../q97.native_datafusion/extended.txt | 61 +- .../q98.native_datafusion/extended.txt | 54 +- .../q1.native_datafusion/extended.txt | 111 +- .../q10.native_datafusion/extended.txt | 72 +- .../q11.native_datafusion/extended.txt | 203 +- .../q12.native_datafusion/extended.txt | 54 +- .../q13.native_datafusion/extended.txt | 88 +- .../q14a.native_datafusion/extended.txt | 1099 ++-- .../q14b.native_datafusion/extended.txt | 796 ++- .../q15.native_datafusion/extended.txt | 66 +- .../q17.native_datafusion/extended.txt | 135 +- .../q18.native_datafusion/extended.txt | 107 +- .../q20.native_datafusion/extended.txt | 54 +- .../q21.native_datafusion/extended.txt | 64 +- .../q22.native_datafusion/extended.txt | 68 +- .../q23a.native_datafusion/extended.txt | 203 +- .../q23b.native_datafusion/extended.txt | 207 +- .../q25.native_datafusion/extended.txt | 135 +- .../q26.native_datafusion/extended.txt | 81 +- .../q27.native_datafusion/extended.txt | 83 +- .../q29.native_datafusion/extended.txt | 140 +- .../q30.native_datafusion/extended.txt | 137 +- .../q31.native_datafusion/extended.txt | 282 +- .../q32.native_datafusion/extended.txt | 93 +- .../q33.native_datafusion/extended.txt | 217 +- .../q34.native_datafusion/extended.txt | 82 +- .../q35.native_datafusion/extended.txt | 72 +- .../q36.native_datafusion/extended.txt | 69 +- .../q37.native_datafusion/extended.txt | 64 +- .../q38.native_datafusion/extended.txt | 133 +- .../q39a.native_datafusion/extended.txt | 134 +- .../q39b.native_datafusion/extended.txt | 134 +- .../q4.native_datafusion/extended.txt | 299 +- .../q40.native_datafusion/extended.txt | 19 +- .../q45.native_datafusion/extended.txt | 66 +- .../q46.native_datafusion/extended.txt | 99 +- .../q47.native_datafusion/extended.txt | 179 +- .../q48.native_datafusion/extended.txt | 77 +- .../q49.native_datafusion/extended.txt | 154 +- .../q5.native_datafusion/extended.txt | 219 +- .../q50.native_datafusion/extended.txt | 77 +- .../q51.native_datafusion/extended.txt | 75 +- .../q53.native_datafusion/extended.txt | 65 +- .../q54.native_datafusion/extended.txt | 210 +- .../q56.native_datafusion/extended.txt | 223 +- .../q57.native_datafusion/extended.txt | 179 +- .../q58.native_datafusion/extended.txt | 258 +- .../q6.native_datafusion/extended.txt | 123 +- .../q60.native_datafusion/extended.txt | 223 +- .../q61.native_datafusion/extended.txt | 188 +- .../q63.native_datafusion/extended.txt | 65 +- .../q64.native_datafusion/extended.txt | 62 +- .../q65.native_datafusion/extended.txt | 115 +- .../q66.native_datafusion/extended.txt | 154 +- .../q67.native_datafusion/extended.txt | 69 +- .../q68.native_datafusion/extended.txt | 99 +- .../q69.native_datafusion/extended.txt | 72 +- .../q7.native_datafusion/extended.txt | 81 +- .../q70.native_datafusion/extended.txt | 77 +- .../q71.native_datafusion/extended.txt | 119 +- .../q72.native_datafusion/extended.txt | 126 +- .../q73.native_datafusion/extended.txt | 82 +- .../q74.native_datafusion/extended.txt | 201 +- .../q75.native_datafusion/extended.txt | 232 +- .../q77.native_datafusion/extended.txt | 257 +- .../q78.native_datafusion/extended.txt | 45 +- .../q79.native_datafusion/extended.txt | 75 +- .../q8.native_datafusion/extended.txt | 106 +- .../q80.native_datafusion/extended.txt | 49 +- .../q81.native_datafusion/extended.txt | 137 +- .../q82.native_datafusion/extended.txt | 64 +- .../q83.native_datafusion/extended.txt | 261 +- .../q85.native_datafusion/extended.txt | 118 +- .../q86.native_datafusion/extended.txt | 56 +- .../q87.native_datafusion/extended.txt | 133 +- .../q89.native_datafusion/extended.txt | 65 +- .../q91.native_datafusion/extended.txt | 104 +- .../q92.native_datafusion/extended.txt | 93 +- .../q97.native_datafusion/extended.txt | 61 +- .../q98.native_datafusion/extended.txt | 54 +- .../q10a.native_datafusion/extended.txt | 129 +- .../q11.native_datafusion/extended.txt | 201 +- .../q12.native_datafusion/extended.txt | 54 +- .../q14.native_datafusion/extended.txt | 796 ++- .../q14a.native_datafusion/extended.txt | 5515 ++++++++--------- .../q18a.native_datafusion/extended.txt | 485 +- .../q20.native_datafusion/extended.txt | 54 +- .../q22.native_datafusion/extended.txt | 42 +- .../q22a.native_datafusion/extended.txt | 344 +- .../q27a.native_datafusion/extended.txt | 223 +- .../q34.native_datafusion/extended.txt | 82 +- .../q35.native_datafusion/extended.txt | 72 +- .../q35a.native_datafusion/extended.txt | 129 +- .../q36a.native_datafusion/extended.txt | 211 +- .../q47.native_datafusion/extended.txt | 179 +- .../q49.native_datafusion/extended.txt | 154 +- .../q51a.native_datafusion/extended.txt | 294 +- .../q57.native_datafusion/extended.txt | 179 +- .../q5a.native_datafusion/extended.txt | 652 +- .../q6.native_datafusion/extended.txt | 123 +- .../q64.native_datafusion/extended.txt | 62 +- .../q67a.native_datafusion/extended.txt | 632 +- .../q70a.native_datafusion/extended.txt | 227 +- .../q72.native_datafusion/extended.txt | 126 +- .../q74.native_datafusion/extended.txt | 201 +- .../q75.native_datafusion/extended.txt | 232 +- .../q77a.native_datafusion/extended.txt | 767 +-- .../q78.native_datafusion/extended.txt | 45 +- .../q80a.native_datafusion/extended.txt | 143 +- .../q86a.native_datafusion/extended.txt | 172 +- .../q98.native_datafusion/extended.txt | 54 +- .../q10a.native_datafusion/extended.txt | 129 +- .../q11.native_datafusion/extended.txt | 201 +- .../q12.native_datafusion/extended.txt | 54 +- .../q14.native_datafusion/extended.txt | 820 +-- .../q14a.native_datafusion/extended.txt | 5515 ++++++++--------- .../q18a.native_datafusion/extended.txt | 485 +- .../q20.native_datafusion/extended.txt | 54 +- .../q22.native_datafusion/extended.txt | 42 +- .../q22a.native_datafusion/extended.txt | 344 +- .../q27a.native_datafusion/extended.txt | 223 +- .../q34.native_datafusion/extended.txt | 82 +- .../q35.native_datafusion/extended.txt | 72 +- .../q35a.native_datafusion/extended.txt | 129 +- .../q36a.native_datafusion/extended.txt | 211 +- .../q47.native_datafusion/extended.txt | 179 +- .../q49.native_datafusion/extended.txt | 154 +- .../q51a.native_datafusion/extended.txt | 294 +- .../q57.native_datafusion/extended.txt | 179 +- .../q5a.native_datafusion/extended.txt | 652 +- .../q6.native_datafusion/extended.txt | 141 +- .../q64.native_datafusion/extended.txt | 62 +- .../q67a.native_datafusion/extended.txt | 632 +- .../q70a.native_datafusion/extended.txt | 227 +- .../q72.native_datafusion/extended.txt | 126 +- .../q74.native_datafusion/extended.txt | 201 +- .../q75.native_datafusion/extended.txt | 232 +- .../q77a.native_datafusion/extended.txt | 767 +-- .../q78.native_datafusion/extended.txt | 45 +- .../q80a.native_datafusion/extended.txt | 143 +- .../q86a.native_datafusion/extended.txt | 172 +- .../q98.native_datafusion/extended.txt | 54 +- .../q10a.native_datafusion/extended.txt | 129 +- .../q11.native_datafusion/extended.txt | 201 +- .../q12.native_datafusion/extended.txt | 54 +- .../q14.native_datafusion/extended.txt | 796 ++- .../q14a.native_datafusion/extended.txt | 5515 ++++++++--------- .../q18a.native_datafusion/extended.txt | 485 +- .../q20.native_datafusion/extended.txt | 54 +- .../q22.native_datafusion/extended.txt | 42 +- .../q22a.native_datafusion/extended.txt | 344 +- .../q27a.native_datafusion/extended.txt | 223 +- .../q34.native_datafusion/extended.txt | 82 +- .../q35.native_datafusion/extended.txt | 72 +- .../q35a.native_datafusion/extended.txt | 129 +- .../q36a.native_datafusion/extended.txt | 211 +- .../q47.native_datafusion/extended.txt | 179 +- .../q49.native_datafusion/extended.txt | 154 +- .../q51a.native_datafusion/extended.txt | 294 +- .../q57.native_datafusion/extended.txt | 179 +- .../q5a.native_datafusion/extended.txt | 652 +- .../q6.native_datafusion/extended.txt | 123 +- .../q64.native_datafusion/extended.txt | 62 +- .../q67a.native_datafusion/extended.txt | 629 +- .../q70a.native_datafusion/extended.txt | 227 +- .../q72.native_datafusion/extended.txt | 126 +- .../q74.native_datafusion/extended.txt | 201 +- .../q75.native_datafusion/extended.txt | 232 +- .../q77a.native_datafusion/extended.txt | 767 +-- .../q78.native_datafusion/extended.txt | 45 +- .../q80a.native_datafusion/extended.txt | 143 +- .../q86a.native_datafusion/extended.txt | 172 +- .../q98.native_datafusion/extended.txt | 54 +- .../sql/comet/CometPlanStabilitySuite.scala | 4 +- 329 files changed, 34984 insertions(+), 36342 deletions(-) diff --git a/spark/src/main/scala/org/apache/spark/sql/comet/CometNativeScanExec.scala b/spark/src/main/scala/org/apache/spark/sql/comet/CometNativeScanExec.scala index 9b134873fc..71b018c728 100644 --- a/spark/src/main/scala/org/apache/spark/sql/comet/CometNativeScanExec.scala +++ b/spark/src/main/scala/org/apache/spark/sql/comet/CometNativeScanExec.scala @@ -134,7 +134,21 @@ case class CometNativeScanExec( if (bucketedScan) { originalPlan.outputPartitioning } else { - UnknownPartitioning(originalPlan.inputRDD.getNumPartitions) + // Use perPartitionData.length instead of originalPlan.inputRDD.getNumPartitions. + // + // originalPlan.inputRDD triggers FileSourceScanExec's full scan pipeline including + // codegen on partition filter expressions. With DPP, this calls + // InSubqueryExec.doGenCode which requires the subquery to have finished — but + // outputPartitioning can be accessed before prepare() runs (e.g., by + // ValidateRequirements during plan validation). + // + // perPartitionData goes through serializedPartitionData, which explicitly resolves + // DPP subqueries (via updateResult()) before accessing file partitions. This is the + // same pattern CometIcebergNativeScanExec uses. + // + // This is also more correct: perPartitionData.length reflects the post-DPP partition + // count, matching what CometExecRDD actually uses in doExecuteColumnar(). + UnknownPartitioning(perPartitionData.length) } } diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q1.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q1.native_datafusion/extended.txt index 91b07a7538..935c103614 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q1.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q1.native_datafusion/extended.txt @@ -1,61 +1,58 @@ -TakeOrderedAndProject -+- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Filter - : : : +- HashAggregate - : : : +- CometNativeColumnarToRow - : : : +- CometColumnarExchange - : : : +- HashAggregate - : : : +- Project - : : : +- BroadcastHashJoin - : : : :- Filter - : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.store_returns [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- Filter - : : +- HashAggregate - : : +- CometNativeColumnarToRow - : : +- CometColumnarExchange - : : +- HashAggregate - : : +- HashAggregate - : : +- CometNativeColumnarToRow - : : +- CometColumnarExchange - : : +- HashAggregate - : : +- Project - : : +- BroadcastHashJoin - : : :- Filter - : : : +- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.store_returns [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : +- ReusedSubquery - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- CometNativeColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.store - +- BroadcastExchange - +- CometNativeColumnarToRow +CometNativeColumnarToRow ++- CometTakeOrderedAndProject + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometHashAggregate + : : : +- CometExchange + : : : +- CometHashAggregate + : : : +- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.store_returns + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometNativeColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometFilter + : : +- CometHashAggregate + : : +- CometExchange + : : +- CometHashAggregate + : : +- CometHashAggregate + : : +- CometExchange + : : +- CometHashAggregate + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.store_returns + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometNativeColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.store + +- CometBroadcastExchange +- CometProject +- CometFilter +- CometNativeScan parquet spark_catalog.default.customer -Comet accelerated 18 out of 49 eligible operators (36%). Final plan contains 10 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 49 out of 53 eligible operators (92%). Final plan contains 3 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q10.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q10.native_datafusion/extended.txt index 40a8ef4a6e..2785c3d0b1 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q10.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q10.native_datafusion/extended.txt @@ -10,46 +10,52 @@ TakeOrderedAndProject : :- Project : : +- Filter : : +- BroadcastHashJoin - : : :- BroadcastHashJoin - : : : :- BroadcastHashJoin - : : : : :- CometNativeColumnarToRow - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.customer - : : : : +- BroadcastExchange - : : : : +- Project - : : : : +- BroadcastHashJoin - : : : : :- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow + : : :- BroadcastHashJoin [COMET: Unsupported join type ExistenceJoin(exists#1)] + : : : :- CometNativeColumnarToRow + : : : : +- CometBroadcastHashJoin + : : : : :- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.customer + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometNativeScan parquet spark_catalog.default.store_sales + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometNativeColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange : : : : +- CometProject : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- BroadcastExchange - : : : +- Project - : : : +- BroadcastHashJoin - : : : :- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- ReusedSubquery - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow + : : : +- CometNativeColumnarToRow + : : : +- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometNativeScan parquet spark_catalog.default.web_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometNativeColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- BroadcastExchange - : : +- Project - : : +- BroadcastHashJoin - : : :- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : +- ReusedSubquery - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow + : : +- CometNativeColumnarToRow + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometNativeColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange : : +- CometProject : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -64,4 +70,4 @@ TakeOrderedAndProject +- CometFilter +- CometNativeScan parquet spark_catalog.default.customer_demographics -Comet accelerated 21 out of 54 eligible operators (38%). Final plan contains 11 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 41 out of 62 eligible operators (66%). Final plan contains 9 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q11.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q11.native_datafusion/extended.txt index 47aa9517cf..6b081dd35b 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q11.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q11.native_datafusion/extended.txt @@ -1,106 +1,99 @@ -TakeOrderedAndProject -+- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Filter - : : : +- HashAggregate - : : : +- CometNativeColumnarToRow - : : : +- CometColumnarExchange - : : : +- HashAggregate - : : : +- Project - : : : +- BroadcastHashJoin - : : : :- Project - : : : : +- BroadcastHashJoin - : : : : :- CometNativeColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.customer - : : : : +- BroadcastExchange - : : : : +- Filter - : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- HashAggregate - : : +- CometNativeColumnarToRow - : : +- CometColumnarExchange - : : +- HashAggregate - : : +- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.customer - : : : +- BroadcastExchange - : : : +- Filter - : : : +- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- Filter - : +- HashAggregate - : +- CometNativeColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.customer - : : +- BroadcastExchange - : : +- Filter - : : +- ColumnarToRow - : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : +- ReusedSubquery - : +- BroadcastExchange - : +- CometNativeColumnarToRow - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.date_dim - +- BroadcastExchange - +- HashAggregate - +- CometNativeColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- CometNativeColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.customer - : +- BroadcastExchange - : +- Filter - : +- ColumnarToRow - : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : +- ReusedSubquery - +- BroadcastExchange - +- CometNativeColumnarToRow - +- CometFilter - +- CometNativeScan parquet spark_catalog.default.date_dim +CometNativeColumnarToRow ++- CometTakeOrderedAndProject + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometHashAggregate + : : : +- CometExchange + : : : +- CometHashAggregate + : : : +- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.customer + : : : : +- CometBroadcastExchange + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometNativeColumnarToRow + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometHashAggregate + : : +- CometExchange + : : +- CometHashAggregate + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.customer + : : : +- CometBroadcastExchange + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometNativeColumnarToRow + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometFilter + : +- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.customer + : : +- CometBroadcastExchange + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.web_sales + : : +- SubqueryBroadcast + : : +- BroadcastExchange + : : +- CometNativeColumnarToRow + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.date_dim + +- CometBroadcastExchange + +- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.customer + : +- CometBroadcastExchange + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.web_sales + : +- SubqueryBroadcast + : +- BroadcastExchange + : +- CometNativeColumnarToRow + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.date_dim + +- CometBroadcastExchange + +- CometFilter + +- CometNativeScan parquet spark_catalog.default.date_dim -Comet accelerated 28 out of 86 eligible operators (32%). Final plan contains 18 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 84 out of 92 eligible operators (91%). Final plan contains 5 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q12.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q12.native_datafusion/extended.txt index 15648408fc..a8c2531a38 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q12.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q12.native_datafusion/extended.txt @@ -3,33 +3,29 @@ TakeOrderedAndProject +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] +- CometNativeColumnarToRow +- CometSort - +- CometColumnarExchange - +- HashAggregate - +- CometNativeColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- Filter - : : +- ColumnarToRow - : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- CometNativeColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.item - +- BroadcastExchange - +- CometNativeColumnarToRow - +- CometProject - +- CometFilter - +- CometNativeScan parquet spark_catalog.default.date_dim + +- CometExchange + +- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.web_sales + : : +- SubqueryBroadcast + : : +- BroadcastExchange + : : +- CometNativeColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.item + +- CometBroadcastExchange + +- CometProject + +- CometFilter + +- CometNativeScan parquet spark_catalog.default.date_dim -Comet accelerated 12 out of 27 eligible operators (44%). Final plan contains 6 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 22 out of 27 eligible operators (81%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q13.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q13.native_datafusion/extended.txt index 9ff51aab85..42ea1c4966 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q13.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q13.native_datafusion/extended.txt @@ -1,48 +1,42 @@ -HashAggregate -+- CometNativeColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- Project - : : : : +- BroadcastHashJoin - : : : : :- Filter - : : : : : +- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.store - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.customer_address - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- CometNativeColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.customer_demographics - +- BroadcastExchange - +- CometNativeColumnarToRow - +- CometFilter - +- CometNativeScan parquet spark_catalog.default.household_demographics +CometNativeColumnarToRow ++- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometNativeColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.store + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.customer_address + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.customer_demographics + +- CometBroadcastExchange + +- CometFilter + +- CometNativeScan parquet spark_catalog.default.household_demographics -Comet accelerated 17 out of 38 eligible operators (44%). Final plan contains 8 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 36 out of 38 eligible operators (94%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q14a.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q14a.native_datafusion/extended.txt index 223593e3d0..0dcd47f57f 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q14a.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q14a.native_datafusion/extended.txt @@ -1,553 +1,554 @@ -TakeOrderedAndProject -+- HashAggregate - +- CometNativeColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Expand - +- Union - :- Project - : +- Filter +CometNativeColumnarToRow ++- CometTakeOrderedAndProject + +- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometExpand + +- CometUnion + :- CometProject + : +- CometFilter : : +- Subquery - : : +- HashAggregate - : : +- CometNativeColumnarToRow - : : +- CometColumnarExchange - : : +- HashAggregate - : : +- Union - : : :- Project - : : : +- BroadcastHashJoin - : : : :- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- ReusedSubquery - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : :- Project - : : : +- BroadcastHashJoin - : : : :- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- ReusedSubquery - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- Project - : : +- BroadcastHashJoin - : : :- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : +- ReusedSubquery - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- HashAggregate - : +- CometNativeColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- BroadcastHashJoin - : : : :- Filter - : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- Project - : : : +- BroadcastHashJoin - : : : :- CometNativeColumnarToRow - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : +- BroadcastExchange - : : : +- BroadcastHashJoin - : : : :- CometNativeColumnarToRow - : : : : +- CometHashAggregate - : : : : +- CometColumnarExchange - : : : : +- HashAggregate - : : : : +- Project - : : : : +- BroadcastHashJoin - : : : : :- Project - : : : : : +- BroadcastHashJoin - : : : : : :- Filter - : : : : : : +- ColumnarToRow - : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometNativeColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : : +- BroadcastExchange - : : : : : +- BroadcastHashJoin - : : : : : :- CometNativeColumnarToRow - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : : +- BroadcastExchange - : : : : : +- Project - : : : : : +- BroadcastHashJoin - : : : : : :- Project - : : : : : : +- BroadcastHashJoin - : : : : : : :- Filter - : : : : : : : +- ColumnarToRow - : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : : : +- ReusedSubquery - : : : : : : +- BroadcastExchange - : : : : : : +- CometNativeColumnarToRow - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : : +- BroadcastExchange - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- Project - : : : +- BroadcastHashJoin - : : : :- Project - : : : : +- BroadcastHashJoin - : : : : :- Filter - : : : : : +- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : +- ReusedSubquery - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- BroadcastHashJoin - : : :- CometNativeColumnarToRow - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.item - : : +- BroadcastExchange - : : +- Project - : : +- BroadcastHashJoin - : : :- CometNativeColumnarToRow - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.item - : : +- BroadcastExchange - : : +- BroadcastHashJoin - : : :- CometNativeColumnarToRow - : : : +- CometHashAggregate - : : : +- CometColumnarExchange - : : : +- HashAggregate - : : : +- Project - : : : +- BroadcastHashJoin - : : : :- Project - : : : : +- BroadcastHashJoin - : : : : :- Filter - : : : : : +- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : +- BroadcastExchange - : : : : +- BroadcastHashJoin - : : : : :- CometNativeColumnarToRow - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : +- BroadcastExchange - : : : : +- Project - : : : : +- BroadcastHashJoin - : : : : :- Project - : : : : : +- BroadcastHashJoin - : : : : : :- Filter - : : : : : : +- ColumnarToRow - : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : : +- ReusedSubquery - : : : : : +- BroadcastExchange - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- Filter - : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- ReusedSubquery - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.item - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- CometNativeColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.date_dim - :- Project - : +- Filter + : : +- CometNativeColumnarToRow + : : +- CometHashAggregate + : : +- CometExchange + : : +- CometHashAggregate + : : +- CometUnion + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometNativeScan parquet spark_catalog.default.store_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometNativeColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometNativeColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometNativeScan parquet spark_catalog.default.web_sales + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometNativeColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometNativeColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : +- CometBroadcastExchange + : : : +- CometBroadcastHashJoin + : : : :- CometHashAggregate + : : : : +- CometExchange + : : : : +- CometHashAggregate + : : : : +- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometProject + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : : : +- SubqueryBroadcast + : : : : : : +- BroadcastExchange + : : : : : : +- CometNativeColumnarToRow + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- CometBroadcastExchange + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : : : +- CometBroadcastExchange + : : : : : +- CometProject + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometProject + : : : : : : +- CometBroadcastHashJoin + : : : : : : :- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : : : : +- SubqueryBroadcast + : : : : : : : +- BroadcastExchange + : : : : : : : +- CometNativeColumnarToRow + : : : : : : : +- CometProject + : : : : : : : +- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : +- CometBroadcastExchange + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : : : +- CometBroadcastExchange + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.web_sales + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometNativeColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.item + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.item + : : +- CometBroadcastExchange + : : +- CometBroadcastHashJoin + : : :- CometHashAggregate + : : : +- CometExchange + : : : +- CometHashAggregate + : : : +- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometNativeColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometBroadcastHashJoin + : : : : :- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometProject + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : : : +- SubqueryBroadcast + : : : : : : +- BroadcastExchange + : : : : : : +- CometNativeColumnarToRow + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- CometBroadcastExchange + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.web_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometNativeColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.item + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.date_dim + :- CometProject + : +- CometFilter : : +- ReusedSubquery - : +- HashAggregate - : +- CometNativeColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- BroadcastHashJoin - : : : :- Filter - : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- ReusedSubquery - : : : +- BroadcastExchange - : : : +- Project - : : : +- BroadcastHashJoin - : : : :- CometNativeColumnarToRow - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : +- BroadcastExchange - : : : +- BroadcastHashJoin - : : : :- CometNativeColumnarToRow - : : : : +- CometHashAggregate - : : : : +- CometColumnarExchange - : : : : +- HashAggregate - : : : : +- Project - : : : : +- BroadcastHashJoin - : : : : :- Project - : : : : : +- BroadcastHashJoin - : : : : : :- Filter - : : : : : : +- ColumnarToRow - : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometNativeColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : : +- BroadcastExchange - : : : : : +- BroadcastHashJoin - : : : : : :- CometNativeColumnarToRow - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : : +- BroadcastExchange - : : : : : +- Project - : : : : : +- BroadcastHashJoin - : : : : : :- Project - : : : : : : +- BroadcastHashJoin - : : : : : : :- Filter - : : : : : : : +- ColumnarToRow - : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : : : +- ReusedSubquery - : : : : : : +- BroadcastExchange - : : : : : : +- CometNativeColumnarToRow - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : : +- BroadcastExchange - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- Project - : : : +- BroadcastHashJoin - : : : :- Project - : : : : +- BroadcastHashJoin - : : : : :- Filter - : : : : : +- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : +- ReusedSubquery - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- BroadcastHashJoin - : : :- CometNativeColumnarToRow - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.item - : : +- BroadcastExchange - : : +- Project - : : +- BroadcastHashJoin - : : :- CometNativeColumnarToRow - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.item - : : +- BroadcastExchange - : : +- BroadcastHashJoin - : : :- CometNativeColumnarToRow - : : : +- CometHashAggregate - : : : +- CometColumnarExchange - : : : +- HashAggregate - : : : +- Project - : : : +- BroadcastHashJoin - : : : :- Project - : : : : +- BroadcastHashJoin - : : : : :- Filter - : : : : : +- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : +- BroadcastExchange - : : : : +- BroadcastHashJoin - : : : : :- CometNativeColumnarToRow - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : +- BroadcastExchange - : : : : +- Project - : : : : +- BroadcastHashJoin - : : : : :- Project - : : : : : +- BroadcastHashJoin - : : : : : :- Filter - : : : : : : +- ColumnarToRow - : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : : +- ReusedSubquery - : : : : : +- BroadcastExchange - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- Filter - : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- ReusedSubquery - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.item - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- CometNativeColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.date_dim - +- Project - +- Filter + : +- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometNativeColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : +- CometBroadcastExchange + : : : +- CometBroadcastHashJoin + : : : :- CometHashAggregate + : : : : +- CometExchange + : : : : +- CometHashAggregate + : : : : +- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometProject + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : : : +- SubqueryBroadcast + : : : : : : +- BroadcastExchange + : : : : : : +- CometNativeColumnarToRow + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- CometBroadcastExchange + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : : : +- CometBroadcastExchange + : : : : : +- CometProject + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometProject + : : : : : : +- CometBroadcastHashJoin + : : : : : : :- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : : : : +- SubqueryBroadcast + : : : : : : : +- BroadcastExchange + : : : : : : : +- CometNativeColumnarToRow + : : : : : : : +- CometProject + : : : : : : : +- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : +- CometBroadcastExchange + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : : : +- CometBroadcastExchange + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.web_sales + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometNativeColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.item + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.item + : : +- CometBroadcastExchange + : : +- CometBroadcastHashJoin + : : :- CometHashAggregate + : : : +- CometExchange + : : : +- CometHashAggregate + : : : +- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometNativeColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometBroadcastHashJoin + : : : : :- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometProject + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : : : +- SubqueryBroadcast + : : : : : : +- BroadcastExchange + : : : : : : +- CometNativeColumnarToRow + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- CometBroadcastExchange + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.web_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometNativeColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.item + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.date_dim + +- CometProject + +- CometFilter : +- ReusedSubquery - +- HashAggregate - +- CometNativeColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- BroadcastHashJoin - : : :- Filter - : : : +- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : +- ReusedSubquery - : : +- BroadcastExchange - : : +- Project - : : +- BroadcastHashJoin - : : :- CometNativeColumnarToRow - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.item - : : +- BroadcastExchange - : : +- BroadcastHashJoin - : : :- CometNativeColumnarToRow - : : : +- CometHashAggregate - : : : +- CometColumnarExchange - : : : +- HashAggregate - : : : +- Project - : : : +- BroadcastHashJoin - : : : :- Project - : : : : +- BroadcastHashJoin - : : : : :- Filter - : : : : : +- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : +- BroadcastExchange - : : : : +- BroadcastHashJoin - : : : : :- CometNativeColumnarToRow - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : +- BroadcastExchange - : : : : +- Project - : : : : +- BroadcastHashJoin - : : : : :- Project - : : : : : +- BroadcastHashJoin - : : : : : :- Filter - : : : : : : +- ColumnarToRow - : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : : +- ReusedSubquery - : : : : : +- BroadcastExchange - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- Filter - : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- ReusedSubquery - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.item - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- BroadcastHashJoin - : :- CometNativeColumnarToRow - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.item - : +- BroadcastExchange - : +- Project - : +- BroadcastHashJoin - : :- CometNativeColumnarToRow - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.item - : +- BroadcastExchange - : +- BroadcastHashJoin - : :- CometNativeColumnarToRow - : : +- CometHashAggregate - : : +- CometColumnarExchange - : : +- HashAggregate - : : +- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- Filter - : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- BroadcastHashJoin - : : : :- CometNativeColumnarToRow - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : +- BroadcastExchange - : : : +- Project - : : : +- BroadcastHashJoin - : : : :- Project - : : : : +- BroadcastHashJoin - : : : : :- Filter - : : : : : +- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : +- ReusedSubquery - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Filter - : : : +- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : +- ReusedSubquery - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.item - : +- BroadcastExchange - : +- CometNativeColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.date_dim - +- BroadcastExchange - +- CometNativeColumnarToRow - +- CometProject - +- CometFilter - +- CometNativeScan parquet spark_catalog.default.date_dim + +- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.web_sales + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometNativeColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.item + : : +- CometBroadcastExchange + : : +- CometBroadcastHashJoin + : : :- CometHashAggregate + : : : +- CometExchange + : : : +- CometHashAggregate + : : : +- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometNativeColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometBroadcastHashJoin + : : : : :- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometProject + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : : : +- SubqueryBroadcast + : : : : : : +- BroadcastExchange + : : : : : : +- CometNativeColumnarToRow + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- CometBroadcastExchange + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.web_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometNativeColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.item + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometBroadcastHashJoin + : :- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.item + : +- CometBroadcastExchange + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.item + : +- CometBroadcastExchange + : +- CometBroadcastHashJoin + : :- CometHashAggregate + : : +- CometExchange + : : +- CometHashAggregate + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometNativeColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometNativeColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.web_sales + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometNativeColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.item + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.date_dim + +- CometBroadcastExchange + +- CometProject + +- CometFilter + +- CometNativeScan parquet spark_catalog.default.date_dim -Comet accelerated 164 out of 458 eligible operators (35%). Final plan contains 93 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 475 out of 526 eligible operators (90%). Final plan contains 26 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q14b.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q14b.native_datafusion/extended.txt index 7921f2fa61..50139381f2 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q14b.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q14b.native_datafusion/extended.txt @@ -1,404 +1,394 @@ -TakeOrderedAndProject -+- BroadcastHashJoin - :- Filter - : : +- Subquery - : : +- HashAggregate - : : +- CometNativeColumnarToRow - : : +- CometColumnarExchange - : : +- HashAggregate - : : +- Union - : : :- Project - : : : +- BroadcastHashJoin - : : : :- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- ReusedSubquery - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : :- Project - : : : +- BroadcastHashJoin - : : : :- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- ReusedSubquery - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- Project - : : +- BroadcastHashJoin - : : :- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : +- ReusedSubquery - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- HashAggregate - : +- CometNativeColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- BroadcastHashJoin - : : : :- Filter - : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : : +- Subquery - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- Project - : : : +- BroadcastHashJoin - : : : :- CometNativeColumnarToRow - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : +- BroadcastExchange - : : : +- BroadcastHashJoin - : : : :- CometNativeColumnarToRow - : : : : +- CometHashAggregate - : : : : +- CometColumnarExchange - : : : : +- HashAggregate - : : : : +- Project - : : : : +- BroadcastHashJoin - : : : : :- Project - : : : : : +- BroadcastHashJoin - : : : : : :- Filter - : : : : : : +- ColumnarToRow - : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometNativeColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : : +- BroadcastExchange - : : : : : +- BroadcastHashJoin - : : : : : :- CometNativeColumnarToRow - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : : +- BroadcastExchange - : : : : : +- Project - : : : : : +- BroadcastHashJoin - : : : : : :- Project - : : : : : : +- BroadcastHashJoin - : : : : : : :- Filter - : : : : : : : +- ColumnarToRow - : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : : : +- ReusedSubquery - : : : : : : +- BroadcastExchange - : : : : : : +- CometNativeColumnarToRow - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : : +- BroadcastExchange - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- Project - : : : +- BroadcastHashJoin - : : : :- Project - : : : : +- BroadcastHashJoin - : : : : :- Filter - : : : : : +- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : +- ReusedSubquery - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- BroadcastHashJoin - : : :- CometNativeColumnarToRow - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.item - : : +- BroadcastExchange - : : +- Project - : : +- BroadcastHashJoin - : : :- CometNativeColumnarToRow - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.item - : : +- BroadcastExchange - : : +- BroadcastHashJoin - : : :- CometNativeColumnarToRow - : : : +- CometHashAggregate - : : : +- CometColumnarExchange - : : : +- HashAggregate - : : : +- Project - : : : +- BroadcastHashJoin - : : : :- Project - : : : : +- BroadcastHashJoin - : : : : :- Filter - : : : : : +- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : +- BroadcastExchange - : : : : +- BroadcastHashJoin - : : : : :- CometNativeColumnarToRow - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : +- BroadcastExchange - : : : : +- Project - : : : : +- BroadcastHashJoin - : : : : :- Project - : : : : : +- BroadcastHashJoin - : : : : : :- Filter - : : : : : : +- ColumnarToRow - : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : : +- ReusedSubquery - : : : : : +- BroadcastExchange - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- Filter - : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- ReusedSubquery - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.item - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- CometNativeColumnarToRow - : +- CometProject - : +- CometFilter - : : +- Subquery - : : +- CometNativeColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- CometNativeScan parquet spark_catalog.default.date_dim - +- BroadcastExchange - +- Filter - : +- ReusedSubquery - +- HashAggregate - +- CometNativeColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- BroadcastHashJoin - : : :- Filter - : : : +- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : : +- Subquery - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- Project - : : +- BroadcastHashJoin - : : :- CometNativeColumnarToRow - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.item - : : +- BroadcastExchange - : : +- BroadcastHashJoin - : : :- CometNativeColumnarToRow - : : : +- CometHashAggregate - : : : +- CometColumnarExchange - : : : +- HashAggregate - : : : +- Project - : : : +- BroadcastHashJoin - : : : :- Project - : : : : +- BroadcastHashJoin - : : : : :- Filter - : : : : : +- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : +- BroadcastExchange - : : : : +- BroadcastHashJoin - : : : : :- CometNativeColumnarToRow - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : +- BroadcastExchange - : : : : +- Project - : : : : +- BroadcastHashJoin - : : : : :- Project - : : : : : +- BroadcastHashJoin - : : : : : :- Filter - : : : : : : +- ColumnarToRow - : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : : +- ReusedSubquery - : : : : : +- BroadcastExchange - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- Filter - : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- ReusedSubquery - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.item - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- BroadcastHashJoin - : :- CometNativeColumnarToRow - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.item - : +- BroadcastExchange - : +- Project - : +- BroadcastHashJoin - : :- CometNativeColumnarToRow - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.item - : +- BroadcastExchange - : +- BroadcastHashJoin - : :- CometNativeColumnarToRow - : : +- CometHashAggregate - : : +- CometColumnarExchange - : : +- HashAggregate - : : +- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- Filter - : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- BroadcastHashJoin - : : : :- CometNativeColumnarToRow - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : +- BroadcastExchange - : : : +- Project - : : : +- BroadcastHashJoin - : : : :- Project - : : : : +- BroadcastHashJoin - : : : : :- Filter - : : : : : +- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : +- ReusedSubquery - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Filter - : : : +- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : +- ReusedSubquery - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.item - : +- BroadcastExchange - : +- CometNativeColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.date_dim - +- BroadcastExchange - +- CometNativeColumnarToRow - +- CometProject - +- CometFilter - : +- Subquery - : +- CometNativeColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.date_dim - +- CometNativeScan parquet spark_catalog.default.date_dim +CometNativeColumnarToRow ++- CometTakeOrderedAndProject + +- CometBroadcastHashJoin + :- CometFilter + : : +- Subquery + : : +- CometNativeColumnarToRow + : : +- CometHashAggregate + : : +- CometExchange + : : +- CometHashAggregate + : : +- CometUnion + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometNativeScan parquet spark_catalog.default.store_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometNativeColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometNativeColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometNativeScan parquet spark_catalog.default.web_sales + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometNativeColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometNativeColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : : +- Subquery + : : : : : +- CometNativeColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : +- CometBroadcastExchange + : : : +- CometBroadcastHashJoin + : : : :- CometHashAggregate + : : : : +- CometExchange + : : : : +- CometHashAggregate + : : : : +- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometProject + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : : : +- SubqueryBroadcast + : : : : : : +- BroadcastExchange + : : : : : : +- CometNativeColumnarToRow + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- CometBroadcastExchange + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : : : +- CometBroadcastExchange + : : : : : +- CometProject + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometProject + : : : : : : +- CometBroadcastHashJoin + : : : : : : :- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : : : : +- SubqueryBroadcast + : : : : : : : +- BroadcastExchange + : : : : : : : +- CometNativeColumnarToRow + : : : : : : : +- CometProject + : : : : : : : +- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : +- CometBroadcastExchange + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : : : +- CometBroadcastExchange + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.web_sales + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometNativeColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.item + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.item + : : +- CometBroadcastExchange + : : +- CometBroadcastHashJoin + : : :- CometHashAggregate + : : : +- CometExchange + : : : +- CometHashAggregate + : : : +- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometNativeColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometBroadcastHashJoin + : : : : :- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometProject + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : : : +- SubqueryBroadcast + : : : : : : +- BroadcastExchange + : : : : : : +- CometNativeColumnarToRow + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- CometBroadcastExchange + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.web_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometNativeColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.item + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : : +- ReusedSubquery + : +- CometNativeScan parquet spark_catalog.default.date_dim + +- CometBroadcastExchange + +- CometFilter + : +- ReusedSubquery + +- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometNativeColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : : +- Subquery + : : : : +- CometNativeColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.item + : : +- CometBroadcastExchange + : : +- CometBroadcastHashJoin + : : :- CometHashAggregate + : : : +- CometExchange + : : : +- CometHashAggregate + : : : +- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometNativeColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometBroadcastHashJoin + : : : : :- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometProject + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : : : +- SubqueryBroadcast + : : : : : : +- BroadcastExchange + : : : : : : +- CometNativeColumnarToRow + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- CometBroadcastExchange + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.web_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometNativeColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.item + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometBroadcastHashJoin + : :- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.item + : +- CometBroadcastExchange + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.item + : +- CometBroadcastExchange + : +- CometBroadcastHashJoin + : :- CometHashAggregate + : : +- CometExchange + : : +- CometHashAggregate + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometNativeColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometNativeColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.web_sales + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometNativeColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.item + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.date_dim + +- CometBroadcastExchange + +- CometProject + +- CometFilter + : +- ReusedSubquery + +- CometNativeScan parquet spark_catalog.default.date_dim -Comet accelerated 128 out of 333 eligible operators (38%). Final plan contains 69 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 331 out of 371 eligible operators (89%). Final plan contains 21 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q15.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q15.native_datafusion/extended.txt index d0d170db53..42432bdc12 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q15.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q15.native_datafusion/extended.txt @@ -1,36 +1,32 @@ -TakeOrderedAndProject -+- HashAggregate - +- CometNativeColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Filter - : : : +- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.customer - : +- BroadcastExchange - : +- CometNativeColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.customer_address - +- BroadcastExchange - +- CometNativeColumnarToRow - +- CometProject - +- CometFilter - +- CometNativeScan parquet spark_catalog.default.date_dim +CometNativeColumnarToRow ++- CometTakeOrderedAndProject + +- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometNativeColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.customer + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.customer_address + +- CometBroadcastExchange + +- CometProject + +- CometFilter + +- CometNativeScan parquet spark_catalog.default.date_dim -Comet accelerated 12 out of 28 eligible operators (42%). Final plan contains 6 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 26 out of 28 eligible operators (92%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q17.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q17.native_datafusion/extended.txt index 15c6bec5dd..f58c012650 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q17.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q17.native_datafusion/extended.txt @@ -1,70 +1,67 @@ -TakeOrderedAndProject -+- HashAggregate - +- CometNativeColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- Project - : : : : +- BroadcastHashJoin - : : : : :- Project - : : : : : +- BroadcastHashJoin - : : : : : :- Project - : : : : : : +- BroadcastHashJoin - : : : : : : :- Filter - : : : : : : : +- ColumnarToRow - : : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : : : +- SubqueryBroadcast - : : : : : : : +- BroadcastExchange - : : : : : : : +- CometNativeColumnarToRow - : : : : : : : +- CometProject - : : : : : : : +- CometFilter - : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : : : +- BroadcastExchange - : : : : : : +- Filter - : : : : : : +- ColumnarToRow - : : : : : : +- Scan parquet spark_catalog.default.store_returns [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometNativeColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : : +- BroadcastExchange - : : : : : +- Filter - : : : : : +- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : +- ReusedSubquery - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- CometNativeColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.store - +- BroadcastExchange - +- CometNativeColumnarToRow - +- CometProject - +- CometFilter - +- CometNativeScan parquet spark_catalog.default.item +CometNativeColumnarToRow ++- CometTakeOrderedAndProject + +- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometProject + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometProject + : : : : : : +- CometBroadcastHashJoin + : : : : : : :- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : : : : +- SubqueryBroadcast + : : : : : : : +- BroadcastExchange + : : : : : : : +- CometNativeColumnarToRow + : : : : : : : +- CometProject + : : : : : : : +- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : +- CometBroadcastExchange + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.store_returns + : : : : : : +- SubqueryBroadcast + : : : : : : +- BroadcastExchange + : : : : : : +- CometNativeColumnarToRow + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- CometBroadcastExchange + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometNativeColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.store + +- CometBroadcastExchange + +- CometProject + +- CometFilter + +- CometNativeScan parquet spark_catalog.default.item -Comet accelerated 22 out of 57 eligible operators (38%). Final plan contains 11 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 55 out of 61 eligible operators (90%). Final plan contains 4 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q18.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q18.native_datafusion/extended.txt index 53fa7b6342..b3485a3421 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q18.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q18.native_datafusion/extended.txt @@ -1,58 +1,51 @@ -TakeOrderedAndProject -+- HashAggregate - +- CometNativeColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Expand - +- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- Project - : : : : +- BroadcastHashJoin - : : : : :- Project - : : : : : +- BroadcastHashJoin - : : : : : :- Filter - : : : : : : +- ColumnarToRow - : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometNativeColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : : +- BroadcastExchange - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.customer_demographics - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.customer - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.customer_demographics - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.customer_address - : +- BroadcastExchange - : +- CometNativeColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.date_dim - +- BroadcastExchange - +- CometNativeColumnarToRow - +- CometProject - +- CometFilter - +- CometNativeScan parquet spark_catalog.default.item +CometNativeColumnarToRow ++- CometTakeOrderedAndProject + +- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometExpand + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometProject + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : : : +- SubqueryBroadcast + : : : : : : +- BroadcastExchange + : : : : : : +- CometNativeColumnarToRow + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- CometBroadcastExchange + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.customer_demographics + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.customer + : : : +- CometBroadcastExchange + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.customer_demographics + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.customer_address + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.date_dim + +- CometBroadcastExchange + +- CometProject + +- CometFilter + +- CometNativeScan parquet spark_catalog.default.item -Comet accelerated 21 out of 47 eligible operators (44%). Final plan contains 9 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 45 out of 47 eligible operators (95%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q20.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q20.native_datafusion/extended.txt index 6634edcbed..c4f54f4289 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q20.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q20.native_datafusion/extended.txt @@ -3,33 +3,29 @@ TakeOrderedAndProject +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] +- CometNativeColumnarToRow +- CometSort - +- CometColumnarExchange - +- HashAggregate - +- CometNativeColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- Filter - : : +- ColumnarToRow - : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- CometNativeColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.item - +- BroadcastExchange - +- CometNativeColumnarToRow - +- CometProject - +- CometFilter - +- CometNativeScan parquet spark_catalog.default.date_dim + +- CometExchange + +- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : +- SubqueryBroadcast + : : +- BroadcastExchange + : : +- CometNativeColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.item + +- CometBroadcastExchange + +- CometProject + +- CometFilter + +- CometNativeScan parquet spark_catalog.default.date_dim -Comet accelerated 12 out of 27 eligible operators (44%). Final plan contains 6 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 22 out of 27 eligible operators (81%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q21.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q21.native_datafusion/extended.txt index 407648ea84..8522eb4ab9 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q21.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q21.native_datafusion/extended.txt @@ -1,35 +1,31 @@ -TakeOrderedAndProject -+- Filter - +- HashAggregate - +- CometNativeColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Filter - : : : +- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.inventory [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.warehouse - : +- BroadcastExchange - : +- CometNativeColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.item - +- BroadcastExchange - +- CometNativeColumnarToRow - +- CometFilter - +- CometNativeScan parquet spark_catalog.default.date_dim +CometNativeColumnarToRow ++- CometTakeOrderedAndProject + +- CometFilter + +- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.inventory + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometNativeColumnarToRow + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.warehouse + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.item + +- CometBroadcastExchange + +- CometFilter + +- CometNativeScan parquet spark_catalog.default.date_dim -Comet accelerated 10 out of 27 eligible operators (37%). Final plan contains 6 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 25 out of 27 eligible operators (92%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q22.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q22.native_datafusion/extended.txt index e1766e64d8..6b78d21629 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q22.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q22.native_datafusion/extended.txt @@ -1,37 +1,33 @@ -TakeOrderedAndProject -+- HashAggregate - +- CometNativeColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Expand - +- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Filter - : : : +- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.inventory [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- CometNativeColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.item - +- BroadcastExchange - +- CometNativeColumnarToRow - +- CometFilter - +- CometNativeScan parquet spark_catalog.default.warehouse +CometNativeColumnarToRow ++- CometTakeOrderedAndProject + +- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometExpand + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.inventory + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometNativeColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.item + +- CometBroadcastExchange + +- CometFilter + +- CometNativeScan parquet spark_catalog.default.warehouse -Comet accelerated 12 out of 29 eligible operators (41%). Final plan contains 6 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 27 out of 29 eligible operators (93%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q23a.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q23a.native_datafusion/extended.txt index 7040e78da1..8a9990d536 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q23a.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q23a.native_datafusion/extended.txt @@ -8,80 +8,71 @@ CometNativeColumnarToRow : :- CometProject : : +- CometSortMergeJoin : : :- CometSort - : : : +- CometColumnarExchange - : : : +- Project - : : : +- BroadcastHashJoin - : : : :- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- Project - : : : +- Filter - : : : +- HashAggregate - : : : +- CometNativeColumnarToRow - : : : +- CometColumnarExchange - : : : +- HashAggregate - : : : +- Project - : : : +- BroadcastHashJoin - : : : :- Project - : : : : +- BroadcastHashJoin - : : : : :- Filter - : : : : : +- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.item + : : : +- CometExchange + : : : +- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometNativeColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometHashAggregate + : : : +- CometExchange + : : : +- CometHashAggregate + : : : +- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometNativeColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.item : : +- CometSort : : +- CometProject : : +- CometFilter : : : +- Subquery - : : : +- HashAggregate - : : : +- CometNativeColumnarToRow - : : : +- CometColumnarExchange - : : : +- HashAggregate - : : : +- HashAggregate - : : : +- CometNativeColumnarToRow - : : : +- CometColumnarExchange - : : : +- HashAggregate - : : : +- Project - : : : +- BroadcastHashJoin - : : : :- Project - : : : : +- BroadcastHashJoin - : : : : :- Filter - : : : : : +- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.customer - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometNativeColumnarToRow + : : : +- CometHashAggregate + : : : +- CometExchange + : : : +- CometHashAggregate + : : : +- CometHashAggregate + : : : +- CometExchange + : : : +- CometHashAggregate + : : : +- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometNativeColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.customer + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- CometHashAggregate : : +- CometExchange : : +- CometHashAggregate @@ -102,41 +93,41 @@ CometNativeColumnarToRow :- CometProject : +- CometSortMergeJoin : :- CometSort - : : +- CometColumnarExchange - : : +- Project - : : +- BroadcastHashJoin - : : :- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : +- ReusedSubquery - : : +- BroadcastExchange - : : +- Project - : : +- Filter - : : +- HashAggregate - : : +- CometNativeColumnarToRow - : : +- CometColumnarExchange - : : +- HashAggregate - : : +- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- Filter - : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.item + : : +- CometExchange + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometNativeScan parquet spark_catalog.default.web_sales + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometNativeColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometHashAggregate + : : +- CometExchange + : : +- CometHashAggregate + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometNativeColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.item : +- CometSort : +- CometProject : +- CometFilter @@ -157,4 +148,4 @@ CometNativeColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.date_dim -Comet accelerated 83 out of 138 eligible operators (60%). Final plan contains 20 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 130 out of 142 eligible operators (91%). Final plan contains 7 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q23b.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q23b.native_datafusion/extended.txt index 188775e7df..52090c668e 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q23b.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q23b.native_datafusion/extended.txt @@ -10,81 +10,72 @@ CometNativeColumnarToRow : : +- CometBroadcastHashJoin : : :- CometSortMergeJoin : : : :- CometSort - : : : : +- CometColumnarExchange - : : : : +- Project - : : : : +- BroadcastHashJoin - : : : : :- Filter - : : : : : +- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : +- BroadcastExchange - : : : : +- Project - : : : : +- Filter - : : : : +- HashAggregate - : : : : +- CometNativeColumnarToRow - : : : : +- CometColumnarExchange - : : : : +- HashAggregate - : : : : +- Project - : : : : +- BroadcastHashJoin - : : : : :- Project - : : : : : +- BroadcastHashJoin - : : : : : :- Filter - : : : : : : +- ColumnarToRow - : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometNativeColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : : +- BroadcastExchange - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : : +- CometExchange + : : : : +- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometNativeColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometHashAggregate + : : : : +- CometExchange + : : : : +- CometHashAggregate + : : : : +- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometProject + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : : : +- SubqueryBroadcast + : : : : : : +- BroadcastExchange + : : : : : : +- CometNativeColumnarToRow + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- CometBroadcastExchange + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.item : : : +- CometSort : : : +- CometProject : : : +- CometFilter : : : : +- Subquery - : : : : +- HashAggregate - : : : : +- CometNativeColumnarToRow - : : : : +- CometColumnarExchange - : : : : +- HashAggregate - : : : : +- HashAggregate - : : : : +- CometNativeColumnarToRow - : : : : +- CometColumnarExchange - : : : : +- HashAggregate - : : : : +- Project - : : : : +- BroadcastHashJoin - : : : : :- Project - : : : : : +- BroadcastHashJoin - : : : : : :- Filter - : : : : : : +- ColumnarToRow - : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometNativeColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : : +- BroadcastExchange - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.customer - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometNativeColumnarToRow + : : : : +- CometHashAggregate + : : : : +- CometExchange + : : : : +- CometHashAggregate + : : : : +- CometHashAggregate + : : : : +- CometExchange + : : : : +- CometHashAggregate + : : : : +- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometProject + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : : : +- SubqueryBroadcast + : : : : : : +- BroadcastExchange + : : : : : : +- CometNativeColumnarToRow + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- CometBroadcastExchange + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.customer + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- CometHashAggregate : : : +- CometExchange : : : +- CometHashAggregate @@ -131,42 +122,42 @@ CometNativeColumnarToRow : +- CometBroadcastHashJoin : :- CometSortMergeJoin : : :- CometSort - : : : +- CometColumnarExchange - : : : +- Project - : : : +- BroadcastHashJoin - : : : :- Filter - : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- ReusedSubquery - : : : +- BroadcastExchange - : : : +- Project - : : : +- Filter - : : : +- HashAggregate - : : : +- CometNativeColumnarToRow - : : : +- CometColumnarExchange - : : : +- HashAggregate - : : : +- Project - : : : +- BroadcastHashJoin - : : : :- Project - : : : : +- BroadcastHashJoin - : : : : :- Filter - : : : : : +- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.item + : : : +- CometExchange + : : : +- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.web_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometNativeColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometHashAggregate + : : : +- CometExchange + : : : +- CometHashAggregate + : : : +- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometNativeColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.item : : +- CometSort : : +- CometProject : : +- CometFilter @@ -209,4 +200,4 @@ CometNativeColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.date_dim -Comet accelerated 131 out of 190 eligible operators (68%). Final plan contains 20 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 180 out of 194 eligible operators (92%). Final plan contains 7 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q25.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q25.native_datafusion/extended.txt index 15c6bec5dd..f58c012650 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q25.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q25.native_datafusion/extended.txt @@ -1,70 +1,67 @@ -TakeOrderedAndProject -+- HashAggregate - +- CometNativeColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- Project - : : : : +- BroadcastHashJoin - : : : : :- Project - : : : : : +- BroadcastHashJoin - : : : : : :- Project - : : : : : : +- BroadcastHashJoin - : : : : : : :- Filter - : : : : : : : +- ColumnarToRow - : : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : : : +- SubqueryBroadcast - : : : : : : : +- BroadcastExchange - : : : : : : : +- CometNativeColumnarToRow - : : : : : : : +- CometProject - : : : : : : : +- CometFilter - : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : : : +- BroadcastExchange - : : : : : : +- Filter - : : : : : : +- ColumnarToRow - : : : : : : +- Scan parquet spark_catalog.default.store_returns [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometNativeColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : : +- BroadcastExchange - : : : : : +- Filter - : : : : : +- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : +- ReusedSubquery - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- CometNativeColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.store - +- BroadcastExchange - +- CometNativeColumnarToRow - +- CometProject - +- CometFilter - +- CometNativeScan parquet spark_catalog.default.item +CometNativeColumnarToRow ++- CometTakeOrderedAndProject + +- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometProject + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometProject + : : : : : : +- CometBroadcastHashJoin + : : : : : : :- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : : : : +- SubqueryBroadcast + : : : : : : : +- BroadcastExchange + : : : : : : : +- CometNativeColumnarToRow + : : : : : : : +- CometProject + : : : : : : : +- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : +- CometBroadcastExchange + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.store_returns + : : : : : : +- SubqueryBroadcast + : : : : : : +- BroadcastExchange + : : : : : : +- CometNativeColumnarToRow + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- CometBroadcastExchange + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometNativeColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.store + +- CometBroadcastExchange + +- CometProject + +- CometFilter + +- CometNativeScan parquet spark_catalog.default.item -Comet accelerated 22 out of 57 eligible operators (38%). Final plan contains 11 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 55 out of 61 eligible operators (90%). Final plan contains 4 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q26.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q26.native_datafusion/extended.txt index ceba8f5828..71fd3bbe6f 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q26.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q26.native_datafusion/extended.txt @@ -1,44 +1,39 @@ -TakeOrderedAndProject -+- HashAggregate - +- CometNativeColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- Filter - : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.customer_demographics - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- CometNativeColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.item - +- BroadcastExchange - +- CometNativeColumnarToRow - +- CometProject - +- CometFilter - +- CometNativeScan parquet spark_catalog.default.promotion +CometNativeColumnarToRow ++- CometTakeOrderedAndProject + +- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometNativeColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.customer_demographics + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.item + +- CometBroadcastExchange + +- CometProject + +- CometFilter + +- CometNativeScan parquet spark_catalog.default.promotion -Comet accelerated 16 out of 35 eligible operators (45%). Final plan contains 7 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 33 out of 35 eligible operators (94%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q27.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q27.native_datafusion/extended.txt index 20f7517d3e..15ab28f784 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q27.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q27.native_datafusion/extended.txt @@ -1,45 +1,40 @@ -TakeOrderedAndProject -+- HashAggregate - +- CometNativeColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Expand - +- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- Filter - : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.customer_demographics - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- CometNativeColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.store - +- BroadcastExchange - +- CometNativeColumnarToRow - +- CometProject - +- CometFilter - +- CometNativeScan parquet spark_catalog.default.item +CometNativeColumnarToRow ++- CometTakeOrderedAndProject + +- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometExpand + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometNativeColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.customer_demographics + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.store + +- CometBroadcastExchange + +- CometProject + +- CometFilter + +- CometNativeScan parquet spark_catalog.default.item -Comet accelerated 16 out of 36 eligible operators (44%). Final plan contains 7 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 34 out of 36 eligible operators (94%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q29.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q29.native_datafusion/extended.txt index bd9fa52aa7..f58c012650 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q29.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q29.native_datafusion/extended.txt @@ -1,75 +1,67 @@ -TakeOrderedAndProject -+- HashAggregate - +- CometNativeColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- Project - : : : : +- BroadcastHashJoin - : : : : :- Project - : : : : : +- BroadcastHashJoin - : : : : : :- Project - : : : : : : +- BroadcastHashJoin - : : : : : : :- Filter - : : : : : : : +- ColumnarToRow - : : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : : : +- SubqueryBroadcast - : : : : : : : +- BroadcastExchange - : : : : : : : +- CometNativeColumnarToRow - : : : : : : : +- CometProject - : : : : : : : +- CometFilter - : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : : : +- BroadcastExchange - : : : : : : +- Filter - : : : : : : +- ColumnarToRow - : : : : : : +- Scan parquet spark_catalog.default.store_returns [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometNativeColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : : +- BroadcastExchange - : : : : : +- Filter - : : : : : +- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- CometNativeColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.store - +- BroadcastExchange - +- CometNativeColumnarToRow - +- CometProject - +- CometFilter - +- CometNativeScan parquet spark_catalog.default.item +CometNativeColumnarToRow ++- CometTakeOrderedAndProject + +- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometProject + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometProject + : : : : : : +- CometBroadcastHashJoin + : : : : : : :- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : : : : +- SubqueryBroadcast + : : : : : : : +- BroadcastExchange + : : : : : : : +- CometNativeColumnarToRow + : : : : : : : +- CometProject + : : : : : : : +- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : +- CometBroadcastExchange + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.store_returns + : : : : : : +- SubqueryBroadcast + : : : : : : +- BroadcastExchange + : : : : : : +- CometNativeColumnarToRow + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- CometBroadcastExchange + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometNativeColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.store + +- CometBroadcastExchange + +- CometProject + +- CometFilter + +- CometNativeScan parquet spark_catalog.default.item -Comet accelerated 25 out of 61 eligible operators (40%). Final plan contains 12 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 55 out of 61 eligible operators (90%). Final plan contains 4 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q30.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q30.native_datafusion/extended.txt index c2e96eb412..18d0a12a03 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q30.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q30.native_datafusion/extended.txt @@ -1,75 +1,70 @@ -TakeOrderedAndProject -+- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Filter - : : : +- HashAggregate - : : : +- CometNativeColumnarToRow - : : : +- CometColumnarExchange - : : : +- HashAggregate - : : : +- Project - : : : +- BroadcastHashJoin - : : : :- Project - : : : : +- BroadcastHashJoin - : : : : :- Filter - : : : : : +- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.web_returns [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.customer_address - : : +- BroadcastExchange - : : +- Filter - : : +- HashAggregate - : : +- CometNativeColumnarToRow - : : +- CometColumnarExchange - : : +- HashAggregate - : : +- HashAggregate - : : +- CometNativeColumnarToRow - : : +- CometColumnarExchange - : : +- HashAggregate - : : +- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- Filter - : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.web_returns [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- ReusedSubquery - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.customer_address - : +- BroadcastExchange - : +- CometNativeColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.customer - +- BroadcastExchange - +- CometNativeColumnarToRow +CometNativeColumnarToRow ++- CometTakeOrderedAndProject + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometHashAggregate + : : : +- CometExchange + : : : +- CometHashAggregate + : : : +- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.web_returns + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometNativeColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.customer_address + : : +- CometBroadcastExchange + : : +- CometFilter + : : +- CometHashAggregate + : : +- CometExchange + : : +- CometHashAggregate + : : +- CometHashAggregate + : : +- CometExchange + : : +- CometHashAggregate + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.web_returns + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometNativeColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.customer_address + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.customer + +- CometBroadcastExchange +- CometProject +- CometFilter +- CometNativeScan parquet spark_catalog.default.customer_address -Comet accelerated 24 out of 61 eligible operators (39%). Final plan contains 12 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 61 out of 65 eligible operators (93%). Final plan contains 3 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q31.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q31.native_datafusion/extended.txt index 03bcf0413e..d747a5243e 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q31.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q31.native_datafusion/extended.txt @@ -1,150 +1,138 @@ CometNativeColumnarToRow +- CometSort - +- CometColumnarExchange - +- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- BroadcastHashJoin - : : : : :- HashAggregate - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometColumnarExchange - : : : : : +- HashAggregate - : : : : : +- Project - : : : : : +- BroadcastHashJoin - : : : : : :- Project - : : : : : : +- BroadcastHashJoin - : : : : : : :- Filter - : : : : : : : +- ColumnarToRow - : : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : : : +- SubqueryBroadcast - : : : : : : : +- BroadcastExchange - : : : : : : : +- CometNativeColumnarToRow - : : : : : : : +- CometFilter - : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : : : +- BroadcastExchange - : : : : : : +- CometNativeColumnarToRow - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : : +- BroadcastExchange - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.customer_address - : : : : +- BroadcastExchange - : : : : +- HashAggregate - : : : : +- CometNativeColumnarToRow - : : : : +- CometColumnarExchange - : : : : +- HashAggregate - : : : : +- Project - : : : : +- BroadcastHashJoin - : : : : :- Project - : : : : : +- BroadcastHashJoin - : : : : : :- Filter - : : : : : : +- ColumnarToRow - : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometNativeColumnarToRow - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : : +- BroadcastExchange - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.customer_address - : : : +- BroadcastExchange - : : : +- HashAggregate - : : : +- CometNativeColumnarToRow - : : : +- CometColumnarExchange - : : : +- HashAggregate - : : : +- Project - : : : +- BroadcastHashJoin - : : : :- Project - : : : : +- BroadcastHashJoin - : : : : :- Filter - : : : : : +- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.customer_address - : : +- BroadcastExchange - : : +- HashAggregate - : : +- CometNativeColumnarToRow - : : +- CometColumnarExchange - : : +- HashAggregate - : : +- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- Filter - : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- ReusedSubquery - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.customer_address - : +- BroadcastExchange - : +- HashAggregate - : +- CometNativeColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Filter - : : : +- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : +- ReusedSubquery - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- CometNativeColumnarToRow - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.customer_address - +- BroadcastExchange - +- HashAggregate - +- CometNativeColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- Filter - : : +- ColumnarToRow - : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : +- ReusedSubquery - : +- BroadcastExchange - : +- CometNativeColumnarToRow - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.date_dim - +- BroadcastExchange - +- CometNativeColumnarToRow - +- CometFilter - +- CometNativeScan parquet spark_catalog.default.customer_address + +- CometExchange + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometBroadcastHashJoin + : : : : :- CometHashAggregate + : : : : : +- CometExchange + : : : : : +- CometHashAggregate + : : : : : +- CometProject + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometProject + : : : : : : +- CometBroadcastHashJoin + : : : : : : :- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : : : : +- SubqueryBroadcast + : : : : : : : +- BroadcastExchange + : : : : : : : +- CometNativeColumnarToRow + : : : : : : : +- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : +- CometBroadcastExchange + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- CometBroadcastExchange + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.customer_address + : : : : +- CometBroadcastExchange + : : : : +- CometHashAggregate + : : : : +- CometExchange + : : : : +- CometHashAggregate + : : : : +- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometProject + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : : : +- SubqueryBroadcast + : : : : : : +- BroadcastExchange + : : : : : : +- CometNativeColumnarToRow + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- CometBroadcastExchange + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.customer_address + : : : +- CometBroadcastExchange + : : : +- CometHashAggregate + : : : +- CometExchange + : : : +- CometHashAggregate + : : : +- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometNativeColumnarToRow + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.customer_address + : : +- CometBroadcastExchange + : : +- CometHashAggregate + : : +- CometExchange + : : +- CometHashAggregate + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.web_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometNativeColumnarToRow + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.customer_address + : +- CometBroadcastExchange + : +- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.web_sales + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometNativeColumnarToRow + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.customer_address + +- CometBroadcastExchange + +- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.web_sales + : : +- SubqueryBroadcast + : : +- BroadcastExchange + : : +- CometNativeColumnarToRow + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.date_dim + +- CometBroadcastExchange + +- CometFilter + +- CometNativeScan parquet spark_catalog.default.customer_address -Comet accelerated 38 out of 120 eligible operators (31%). Final plan contains 28 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 117 out of 129 eligible operators (90%). Final plan contains 7 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q32.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q32.native_datafusion/extended.txt index f7096f5c3b..5d009dea2d 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q32.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q32.native_datafusion/extended.txt @@ -1,48 +1,47 @@ -HashAggregate -+- CometNativeColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Filter - : : : +- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.item - : +- BroadcastExchange - : +- Filter - : +- HashAggregate - : +- CometNativeColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- Filter - : : +- ColumnarToRow - : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : +- ReusedSubquery - : +- BroadcastExchange - : +- CometNativeColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.date_dim - +- BroadcastExchange - +- CometNativeColumnarToRow - +- CometProject - +- CometFilter - +- CometNativeScan parquet spark_catalog.default.date_dim +CometNativeColumnarToRow ++- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometNativeColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.item + : +- CometBroadcastExchange + : +- CometFilter + : +- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : +- SubqueryBroadcast + : : +- BroadcastExchange + : : +- CometNativeColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.date_dim + +- CometBroadcastExchange + +- CometProject + +- CometFilter + +- CometNativeScan parquet spark_catalog.default.date_dim -Comet accelerated 14 out of 38 eligible operators (36%). Final plan contains 8 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 38 out of 42 eligible operators (90%). Final plan contains 3 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q33.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q33.native_datafusion/extended.txt index 38b3efb123..6afaa68137 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q33.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q33.native_datafusion/extended.txt @@ -1,112 +1,107 @@ -TakeOrderedAndProject -+- HashAggregate - +- CometNativeColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Union - :- HashAggregate - : +- CometNativeColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- Filter - : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.customer_address - : +- BroadcastExchange - : +- CometNativeColumnarToRow - : +- CometBroadcastHashJoin - : :- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.item - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.item - :- HashAggregate - : +- CometNativeColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- Filter - : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- ReusedSubquery - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.customer_address - : +- BroadcastExchange - : +- CometNativeColumnarToRow - : +- CometBroadcastHashJoin - : :- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.item - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.item - +- HashAggregate - +- CometNativeColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Filter - : : : +- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : +- ReusedSubquery - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- CometNativeColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.customer_address - +- BroadcastExchange - +- CometNativeColumnarToRow - +- CometBroadcastHashJoin - :- CometFilter - : +- CometNativeScan parquet spark_catalog.default.item - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometNativeScan parquet spark_catalog.default.item +CometNativeColumnarToRow ++- CometTakeOrderedAndProject + +- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometUnion + :- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometNativeColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.customer_address + : +- CometBroadcastExchange + : +- CometBroadcastHashJoin + : :- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.item + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.item + :- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometNativeColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.customer_address + : +- CometBroadcastExchange + : +- CometBroadcastHashJoin + : :- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.item + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.item + +- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.web_sales + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometNativeColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.customer_address + +- CometBroadcastExchange + +- CometBroadcastHashJoin + :- CometFilter + : +- CometNativeScan parquet spark_catalog.default.item + +- CometBroadcastExchange + +- CometProject + +- CometFilter + +- CometNativeScan parquet spark_catalog.default.item -Comet accelerated 46 out of 93 eligible operators (49%). Final plan contains 17 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 95 out of 101 eligible operators (94%). Final plan contains 4 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q34.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q34.native_datafusion/extended.txt index 40230582bb..1b61be2937 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q34.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q34.native_datafusion/extended.txt @@ -1,47 +1,41 @@ CometNativeColumnarToRow +- CometSort - +- CometColumnarExchange - +- Project - +- BroadcastHashJoin - :- Filter - : +- HashAggregate - : +- CometNativeColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- Filter - : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.store - : +- BroadcastExchange - : +- CometNativeColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.household_demographics - +- BroadcastExchange - +- CometNativeColumnarToRow - +- CometProject - +- CometFilter - +- CometNativeScan parquet spark_catalog.default.customer + +- CometExchange + +- CometProject + +- CometBroadcastHashJoin + :- CometFilter + : +- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometNativeColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.store + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.household_demographics + +- CometBroadcastExchange + +- CometProject + +- CometFilter + +- CometNativeScan parquet spark_catalog.default.customer -Comet accelerated 18 out of 37 eligible operators (48%). Final plan contains 8 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 35 out of 37 eligible operators (94%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q35.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q35.native_datafusion/extended.txt index 40a8ef4a6e..2785c3d0b1 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q35.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q35.native_datafusion/extended.txt @@ -10,46 +10,52 @@ TakeOrderedAndProject : :- Project : : +- Filter : : +- BroadcastHashJoin - : : :- BroadcastHashJoin - : : : :- BroadcastHashJoin - : : : : :- CometNativeColumnarToRow - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.customer - : : : : +- BroadcastExchange - : : : : +- Project - : : : : +- BroadcastHashJoin - : : : : :- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow + : : :- BroadcastHashJoin [COMET: Unsupported join type ExistenceJoin(exists#1)] + : : : :- CometNativeColumnarToRow + : : : : +- CometBroadcastHashJoin + : : : : :- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.customer + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometNativeScan parquet spark_catalog.default.store_sales + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometNativeColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange : : : : +- CometProject : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- BroadcastExchange - : : : +- Project - : : : +- BroadcastHashJoin - : : : :- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- ReusedSubquery - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow + : : : +- CometNativeColumnarToRow + : : : +- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometNativeScan parquet spark_catalog.default.web_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometNativeColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- BroadcastExchange - : : +- Project - : : +- BroadcastHashJoin - : : :- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : +- ReusedSubquery - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow + : : +- CometNativeColumnarToRow + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometNativeColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange : : +- CometProject : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -64,4 +70,4 @@ TakeOrderedAndProject +- CometFilter +- CometNativeScan parquet spark_catalog.default.customer_demographics -Comet accelerated 21 out of 54 eligible operators (38%). Final plan contains 11 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 41 out of 62 eligible operators (66%). Final plan contains 9 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q36.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q36.native_datafusion/extended.txt index aa40ed68cc..05ce793d8d 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q36.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q36.native_datafusion/extended.txt @@ -3,41 +3,36 @@ TakeOrderedAndProject +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] +- CometNativeColumnarToRow +- CometSort - +- CometColumnarExchange - +- HashAggregate - +- CometNativeColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Expand - +- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Filter - : : : +- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- CometNativeColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.item - +- BroadcastExchange - +- CometNativeColumnarToRow - +- CometProject - +- CometFilter - +- CometNativeScan parquet spark_catalog.default.store + +- CometExchange + +- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometExpand + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometNativeColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.item + +- CometBroadcastExchange + +- CometProject + +- CometFilter + +- CometNativeScan parquet spark_catalog.default.store -Comet accelerated 15 out of 34 eligible operators (44%). Final plan contains 7 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 29 out of 34 eligible operators (85%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q37.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q37.native_datafusion/extended.txt index 296b2c8117..16e3a70d9b 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q37.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q37.native_datafusion/extended.txt @@ -1,38 +1,34 @@ CometNativeColumnarToRow +- CometTakeOrderedAndProject +- CometHashAggregate - +- CometColumnarExchange - +- HashAggregate - +- Project - +- BroadcastHashJoin - :- BroadcastExchange - : +- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.item - : : +- BroadcastExchange - : : +- Project - : : +- Filter - : : +- ColumnarToRow - : : +- Scan parquet spark_catalog.default.inventory [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- CometNativeColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.date_dim - +- CometNativeColumnarToRow - +- CometProject - +- CometFilter - +- CometNativeScan parquet spark_catalog.default.catalog_sales + +- CometExchange + +- CometHashAggregate + +- CometProject + +- CometBroadcastHashJoin + :- CometBroadcastExchange + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.item + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.inventory + : : +- SubqueryBroadcast + : : +- BroadcastExchange + : : +- CometNativeColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.date_dim + +- CometProject + +- CometFilter + +- CometNativeScan parquet spark_catalog.default.catalog_sales -Comet accelerated 15 out of 30 eligible operators (50%). Final plan contains 6 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 28 out of 30 eligible operators (93%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q38.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q38.native_datafusion/extended.txt index 6d9f5c7a17..29422273c5 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q38.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q38.native_datafusion/extended.txt @@ -6,74 +6,75 @@ CometNativeColumnarToRow +- CometBroadcastHashJoin :- CometBroadcastHashJoin : :- CometHashAggregate - : : +- CometColumnarExchange - : : +- HashAggregate - : : +- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- Filter - : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.customer + : : +- CometExchange + : : +- CometHashAggregate + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometNativeColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.customer : +- CometBroadcastExchange : +- CometHashAggregate - : +- CometColumnarExchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Filter - : : : +- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : +- ReusedSubquery - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- CometNativeColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.customer + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometNativeColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.customer +- CometBroadcastExchange +- CometHashAggregate - +- CometColumnarExchange - +- HashAggregate - +- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- Filter - : : +- ColumnarToRow - : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : +- ReusedSubquery - : +- BroadcastExchange - : +- CometNativeColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.date_dim - +- BroadcastExchange - +- CometNativeColumnarToRow - +- CometProject - +- CometFilter - +- CometNativeScan parquet spark_catalog.default.customer + +- CometExchange + +- CometHashAggregate + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.web_sales + : : +- SubqueryBroadcast + : : +- BroadcastExchange + : : +- CometNativeColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.date_dim + +- CometBroadcastExchange + +- CometProject + +- CometFilter + +- CometNativeScan parquet spark_catalog.default.customer -Comet accelerated 35 out of 66 eligible operators (53%). Final plan contains 11 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 68 out of 74 eligible operators (91%). Final plan contains 4 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q39a.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q39a.native_datafusion/extended.txt index d9cec17b28..eba1ff4dcd 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q39a.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q39a.native_datafusion/extended.txt @@ -1,75 +1,65 @@ CometNativeColumnarToRow +- CometSort - +- CometColumnarExchange - +- BroadcastHashJoin - :- Project - : +- Filter - : +- HashAggregate - : +- CometNativeColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- Filter - : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.inventory [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.item - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.warehouse - : +- BroadcastExchange - : +- CometNativeColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.date_dim - +- BroadcastExchange - +- Project - +- Filter - +- HashAggregate - +- CometNativeColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Filter - : : : +- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.inventory [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.item - : +- BroadcastExchange - : +- CometNativeColumnarToRow - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.warehouse - +- BroadcastExchange - +- CometNativeColumnarToRow - +- CometProject - +- CometFilter - +- CometNativeScan parquet spark_catalog.default.date_dim + +- CometExchange + +- CometBroadcastHashJoin + :- CometProject + : +- CometFilter + : +- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.inventory + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometNativeColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.item + : : +- CometBroadcastExchange + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.warehouse + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.date_dim + +- CometBroadcastExchange + +- CometProject + +- CometFilter + +- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.inventory + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometNativeColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.item + : +- CometBroadcastExchange + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.warehouse + +- CometBroadcastExchange + +- CometProject + +- CometFilter + +- CometNativeScan parquet spark_catalog.default.date_dim -Comet accelerated 24 out of 60 eligible operators (40%). Final plan contains 13 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 56 out of 60 eligible operators (93%). Final plan contains 3 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q39b.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q39b.native_datafusion/extended.txt index d9cec17b28..eba1ff4dcd 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q39b.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q39b.native_datafusion/extended.txt @@ -1,75 +1,65 @@ CometNativeColumnarToRow +- CometSort - +- CometColumnarExchange - +- BroadcastHashJoin - :- Project - : +- Filter - : +- HashAggregate - : +- CometNativeColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- Filter - : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.inventory [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.item - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.warehouse - : +- BroadcastExchange - : +- CometNativeColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.date_dim - +- BroadcastExchange - +- Project - +- Filter - +- HashAggregate - +- CometNativeColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Filter - : : : +- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.inventory [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.item - : +- BroadcastExchange - : +- CometNativeColumnarToRow - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.warehouse - +- BroadcastExchange - +- CometNativeColumnarToRow - +- CometProject - +- CometFilter - +- CometNativeScan parquet spark_catalog.default.date_dim + +- CometExchange + +- CometBroadcastHashJoin + :- CometProject + : +- CometFilter + : +- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.inventory + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometNativeColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.item + : : +- CometBroadcastExchange + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.warehouse + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.date_dim + +- CometBroadcastExchange + +- CometProject + +- CometFilter + +- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.inventory + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometNativeColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.item + : +- CometBroadcastExchange + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.warehouse + +- CometBroadcastExchange + +- CometProject + +- CometFilter + +- CometNativeScan parquet spark_catalog.default.date_dim -Comet accelerated 24 out of 60 eligible operators (40%). Final plan contains 13 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 56 out of 60 eligible operators (93%). Final plan contains 3 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q4.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q4.native_datafusion/extended.txt index 737217ff91..b2914584b4 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q4.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q4.native_datafusion/extended.txt @@ -1,154 +1,147 @@ -TakeOrderedAndProject -+- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- BroadcastHashJoin - : : : : :- Filter - : : : : : +- HashAggregate - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometColumnarExchange - : : : : : +- HashAggregate - : : : : : +- Project - : : : : : +- BroadcastHashJoin - : : : : : :- Project - : : : : : : +- BroadcastHashJoin - : : : : : : :- CometNativeColumnarToRow - : : : : : : : +- CometProject - : : : : : : : +- CometFilter - : : : : : : : +- CometNativeScan parquet spark_catalog.default.customer - : : : : : : +- BroadcastExchange - : : : : : : +- Filter - : : : : : : +- ColumnarToRow - : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometNativeColumnarToRow - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : : +- BroadcastExchange - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : +- BroadcastExchange - : : : : +- HashAggregate - : : : : +- CometNativeColumnarToRow - : : : : +- CometColumnarExchange - : : : : +- HashAggregate - : : : : +- Project - : : : : +- BroadcastHashJoin - : : : : :- Project - : : : : : +- BroadcastHashJoin - : : : : : :- CometNativeColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.customer - : : : : : +- BroadcastExchange - : : : : : +- Filter - : : : : : +- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- Filter - : : : +- HashAggregate - : : : +- CometNativeColumnarToRow - : : : +- CometColumnarExchange - : : : +- HashAggregate - : : : +- Project - : : : +- BroadcastHashJoin - : : : :- Project - : : : : +- BroadcastHashJoin - : : : : :- CometNativeColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.customer - : : : : +- BroadcastExchange - : : : : +- Filter - : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- ReusedSubquery - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- HashAggregate - : : +- CometNativeColumnarToRow - : : +- CometColumnarExchange - : : +- HashAggregate - : : +- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.customer - : : : +- BroadcastExchange - : : : +- Filter - : : : +- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : +- ReusedSubquery - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- Filter - : +- HashAggregate - : +- CometNativeColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.customer - : : +- BroadcastExchange - : : +- Filter - : : +- ColumnarToRow - : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : +- ReusedSubquery - : +- BroadcastExchange - : +- CometNativeColumnarToRow - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.date_dim - +- BroadcastExchange - +- HashAggregate - +- CometNativeColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- CometNativeColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.customer - : +- BroadcastExchange - : +- Filter - : +- ColumnarToRow - : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : +- ReusedSubquery - +- BroadcastExchange - +- CometNativeColumnarToRow - +- CometFilter - +- CometNativeScan parquet spark_catalog.default.date_dim +CometNativeColumnarToRow ++- CometTakeOrderedAndProject + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometBroadcastHashJoin + : : : : :- CometFilter + : : : : : +- CometHashAggregate + : : : : : +- CometExchange + : : : : : +- CometHashAggregate + : : : : : +- CometProject + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometProject + : : : : : : +- CometBroadcastHashJoin + : : : : : : :- CometProject + : : : : : : : +- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.customer + : : : : : : +- CometBroadcastExchange + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : : : +- SubqueryBroadcast + : : : : : : +- BroadcastExchange + : : : : : : +- CometNativeColumnarToRow + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- CometBroadcastExchange + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometHashAggregate + : : : : +- CometExchange + : : : : +- CometHashAggregate + : : : : +- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometProject + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.customer + : : : : : +- CometBroadcastExchange + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometNativeColumnarToRow + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometFilter + : : : +- CometHashAggregate + : : : +- CometExchange + : : : +- CometHashAggregate + : : : +- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.customer + : : : : +- CometBroadcastExchange + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometNativeColumnarToRow + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometHashAggregate + : : +- CometExchange + : : +- CometHashAggregate + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.customer + : : : +- CometBroadcastExchange + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometNativeColumnarToRow + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometFilter + : +- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.customer + : : +- CometBroadcastExchange + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.web_sales + : : +- SubqueryBroadcast + : : +- BroadcastExchange + : : +- CometNativeColumnarToRow + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.date_dim + +- CometBroadcastExchange + +- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.customer + : +- CometBroadcastExchange + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.web_sales + : +- SubqueryBroadcast + : +- BroadcastExchange + : +- CometNativeColumnarToRow + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.date_dim + +- CometBroadcastExchange + +- CometFilter + +- CometNativeScan parquet spark_catalog.default.date_dim -Comet accelerated 40 out of 126 eligible operators (31%). Final plan contains 26 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 126 out of 138 eligible operators (91%). Final plan contains 7 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q40.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q40.native_datafusion/extended.txt index f1d98d3059..9b00a2c6a4 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q40.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q40.native_datafusion/extended.txt @@ -12,15 +12,14 @@ CometNativeColumnarToRow : : :- CometProject : : : +- CometSortMergeJoin : : : :- CometSort - : : : : +- CometColumnarExchange - : : : : +- Filter - : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometExchange + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometNativeColumnarToRow + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- CometSort : : : +- CometExchange : : : +- CometProject @@ -38,4 +37,4 @@ CometNativeColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.date_dim -Comet accelerated 32 out of 36 eligible operators (88%). Final plan contains 3 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 34 out of 36 eligible operators (94%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q45.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q45.native_datafusion/extended.txt index 5b686ba177..7292a238d5 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q45.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q45.native_datafusion/extended.txt @@ -5,40 +5,36 @@ TakeOrderedAndProject +- HashAggregate +- Project +- Filter - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- Project - : : : : +- BroadcastHashJoin - : : : : :- Filter - : : : : : +- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.customer - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.customer_address - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- CometNativeColumnarToRow + +- BroadcastHashJoin [COMET: Unsupported join type ExistenceJoin(exists#1)] + :- CometNativeColumnarToRow + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.web_sales + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometNativeColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.customer + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.customer_address + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange : +- CometProject : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.item @@ -48,4 +44,4 @@ TakeOrderedAndProject +- CometFilter +- CometNativeScan parquet spark_catalog.default.item -Comet accelerated 18 out of 41 eligible operators (43%). Final plan contains 8 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 32 out of 41 eligible operators (78%). Final plan contains 4 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q46.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q46.native_datafusion/extended.txt index cebcd85144..dbdaaa275a 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q46.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q46.native_datafusion/extended.txt @@ -1,56 +1,49 @@ -TakeOrderedAndProject -+- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- HashAggregate - : : +- CometNativeColumnarToRow - : : +- CometColumnarExchange - : : +- HashAggregate - : : +- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- Project - : : : : +- BroadcastHashJoin - : : : : :- Project - : : : : : +- BroadcastHashJoin - : : : : : :- Filter - : : : : : : +- ColumnarToRow - : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometNativeColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : : +- BroadcastExchange - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.store - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.household_demographics - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.customer_address - : +- BroadcastExchange - : +- CometNativeColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.customer - +- BroadcastExchange - +- CometNativeColumnarToRow +CometNativeColumnarToRow ++- CometTakeOrderedAndProject + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometHashAggregate + : : +- CometExchange + : : +- CometHashAggregate + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometProject + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : : : +- SubqueryBroadcast + : : : : : : +- BroadcastExchange + : : : : : : +- CometNativeColumnarToRow + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- CometBroadcastExchange + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.store + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.household_demographics + : : +- CometBroadcastExchange + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.customer_address + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.customer + +- CometBroadcastExchange +- CometFilter +- CometNativeScan parquet spark_catalog.default.customer_address -Comet accelerated 20 out of 45 eligible operators (44%). Final plan contains 9 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 43 out of 45 eligible operators (95%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q47.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q47.native_datafusion/extended.txt index ab44bbef5c..0886059452 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q47.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q47.native_datafusion/extended.txt @@ -10,111 +10,96 @@ TakeOrderedAndProject : : +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] : : +- CometNativeColumnarToRow : : +- CometSort - : : +- CometColumnarExchange - : : +- HashAggregate - : : +- CometNativeColumnarToRow - : : +- CometColumnarExchange - : : +- HashAggregate - : : +- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- Project - : : : : +- BroadcastHashJoin - : : : : :- CometNativeColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : +- BroadcastExchange - : : : : +- Filter - : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.store + : : +- CometExchange + : : +- CometHashAggregate + : : +- CometExchange + : : +- CometHashAggregate + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : : +- CometBroadcastExchange + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometNativeColumnarToRow + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.store : +- BroadcastExchange : +- Project : +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] : +- CometNativeColumnarToRow : +- CometSort - : +- CometColumnarExchange - : +- HashAggregate - : +- CometNativeColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : +- BroadcastExchange - : : : +- Filter - : : : +- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- CometNativeColumnarToRow - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.store + : +- CometExchange + : +- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : +- CometBroadcastExchange + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometNativeColumnarToRow + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.store +- BroadcastExchange +- Project +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] +- CometNativeColumnarToRow +- CometSort - +- CometColumnarExchange - +- HashAggregate - +- CometNativeColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.item - : : +- BroadcastExchange - : : +- Filter - : : +- ColumnarToRow - : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- CometNativeColumnarToRow - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.date_dim - +- BroadcastExchange - +- CometNativeColumnarToRow - +- CometFilter - +- CometNativeScan parquet spark_catalog.default.store + +- CometExchange + +- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.item + : : +- CometBroadcastExchange + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : +- SubqueryBroadcast + : : +- BroadcastExchange + : : +- CometNativeColumnarToRow + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.date_dim + +- CometBroadcastExchange + +- CometFilter + +- CometNativeScan parquet spark_catalog.default.store -Comet accelerated 36 out of 97 eligible operators (37%). Final plan contains 21 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 75 out of 97 eligible operators (77%). Final plan contains 6 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q48.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q48.native_datafusion/extended.txt index 5e6a26863a..75a1c725c2 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q48.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q48.native_datafusion/extended.txt @@ -1,42 +1,37 @@ -HashAggregate -+- CometNativeColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- Filter - : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.store - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.customer_demographics - : +- BroadcastExchange - : +- CometNativeColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.customer_address - +- BroadcastExchange - +- CometNativeColumnarToRow - +- CometProject - +- CometFilter - +- CometNativeScan parquet spark_catalog.default.date_dim +CometNativeColumnarToRow ++- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometNativeColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.store + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.customer_demographics + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.customer_address + +- CometBroadcastExchange + +- CometProject + +- CometFilter + +- CometNativeScan parquet spark_catalog.default.date_dim -Comet accelerated 15 out of 33 eligible operators (45%). Final plan contains 7 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 31 out of 33 eligible operators (93%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q49.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q49.native_datafusion/extended.txt index 0090739d55..6d7dadcde9 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q49.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q49.native_datafusion/extended.txt @@ -11,35 +11,31 @@ CometNativeColumnarToRow : +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] : +- CometNativeColumnarToRow : +- CometSort - : +- CometColumnarExchange - : +- HashAggregate - : +- CometNativeColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- BroadcastExchange - : : : +- Project - : : : +- Filter - : : : +- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- CometNativeColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.web_returns - : +- BroadcastExchange - : +- CometNativeColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometExchange + : +- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.web_sales + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometNativeColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.web_returns + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.date_dim :- Project : +- Filter : +- Window @@ -47,30 +43,31 @@ CometNativeColumnarToRow : +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] : +- CometNativeColumnarToRow : +- CometSort - : +- CometColumnarExchange - : +- HashAggregate - : +- CometNativeColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- BroadcastExchange - : : : +- Project - : : : +- Filter - : : : +- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : +- ReusedSubquery - : : +- CometNativeColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.catalog_returns - : +- BroadcastExchange - : +- CometNativeColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometExchange + : +- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometNativeColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.catalog_returns + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.date_dim +- Project +- Filter +- Window @@ -78,29 +75,30 @@ CometNativeColumnarToRow +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] +- CometNativeColumnarToRow +- CometSort - +- CometColumnarExchange - +- HashAggregate - +- CometNativeColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- BroadcastExchange - : : +- Project - : : +- Filter - : : +- ColumnarToRow - : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : +- ReusedSubquery - : +- CometNativeColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.store_returns - +- BroadcastExchange - +- CometNativeColumnarToRow - +- CometProject - +- CometFilter - +- CometNativeScan parquet spark_catalog.default.date_dim + +- CometExchange + +- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : +- SubqueryBroadcast + : : +- BroadcastExchange + : : +- CometNativeColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.store_returns + +- CometBroadcastExchange + +- CometProject + +- CometFilter + +- CometNativeScan parquet spark_catalog.default.date_dim -Comet accelerated 33 out of 87 eligible operators (37%). Final plan contains 17 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 72 out of 95 eligible operators (75%). Final plan contains 7 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q5.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q5.native_datafusion/extended.txt index 15ad014521..719229a9a3 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q5.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q5.native_datafusion/extended.txt @@ -1,106 +1,115 @@ -TakeOrderedAndProject -+- HashAggregate - +- CometNativeColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Expand - +- Union - :- HashAggregate - : +- CometNativeColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Union - : : : :- Project - : : : : +- Filter - : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- Project - : : : +- Filter - : : : +- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.store_returns [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : +- ReusedSubquery - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- CometNativeColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.store - :- HashAggregate - : +- CometNativeColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Union - : : : :- Project - : : : : +- Filter - : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- ReusedSubquery - : : : +- Project - : : : +- Filter - : : : +- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.catalog_returns [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : +- ReusedSubquery - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- CometNativeColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.catalog_page - +- HashAggregate - +- CometNativeColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- Union - : : :- Project - : : : +- Filter - : : : +- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : +- ReusedSubquery - : : +- Project - : : +- BroadcastHashJoin - : : :- BroadcastExchange - : : : +- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.web_returns [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : +- ReusedSubquery - : : +- CometNativeColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.web_sales - : +- BroadcastExchange - : +- CometNativeColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.date_dim - +- BroadcastExchange - +- CometNativeColumnarToRow - +- CometProject - +- CometFilter - +- CometNativeScan parquet spark_catalog.default.web_site +CometNativeColumnarToRow ++- CometTakeOrderedAndProject + +- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometExpand + +- CometUnion + :- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometUnion + : : : :- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometNativeColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.store_returns + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometNativeColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.store + :- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometUnion + : : : :- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometNativeColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.catalog_returns + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometNativeColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.catalog_page + +- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometUnion + : : :- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.web_sales + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometNativeColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometBroadcastExchange + : : : +- CometNativeScan parquet spark_catalog.default.web_returns + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometNativeColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.web_sales + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.date_dim + +- CometBroadcastExchange + +- CometProject + +- CometFilter + +- CometNativeScan parquet spark_catalog.default.web_site -Comet accelerated 28 out of 86 eligible operators (32%). Final plan contains 18 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 94 out of 106 eligible operators (88%). Final plan contains 7 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q50.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q50.native_datafusion/extended.txt index ab41730455..c013381de0 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q50.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q50.native_datafusion/extended.txt @@ -1,42 +1,37 @@ -TakeOrderedAndProject -+- HashAggregate - +- CometNativeColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- CometNativeColumnarToRow - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : +- BroadcastExchange - : : : +- Filter - : : : +- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.store_returns [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.store - : +- BroadcastExchange - : +- CometNativeColumnarToRow - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.date_dim - +- BroadcastExchange - +- CometNativeColumnarToRow - +- CometProject - +- CometFilter - +- CometNativeScan parquet spark_catalog.default.date_dim +CometNativeColumnarToRow ++- CometTakeOrderedAndProject + +- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : +- CometBroadcastExchange + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.store_returns + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometNativeColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.store + : +- CometBroadcastExchange + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.date_dim + +- CometBroadcastExchange + +- CometProject + +- CometFilter + +- CometNativeScan parquet spark_catalog.default.date_dim -Comet accelerated 14 out of 33 eligible operators (42%). Final plan contains 7 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 31 out of 33 eligible operators (93%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q51.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q51.native_datafusion/extended.txt index db4e38a1bb..425e772118 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q51.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q51.native_datafusion/extended.txt @@ -12,48 +12,47 @@ TakeOrderedAndProject : +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] : +- CometNativeColumnarToRow : +- CometSort - : +- CometColumnarExchange - : +- HashAggregate - : +- CometNativeColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- Filter - : : +- ColumnarToRow - : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- CometNativeColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometExchange + : +- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.web_sales + : : +- SubqueryBroadcast + : : +- BroadcastExchange + : : +- CometNativeColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.date_dim +- CometSort +- CometColumnarExchange +- Project +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] +- CometNativeColumnarToRow +- CometSort - +- CometColumnarExchange - +- HashAggregate - +- CometNativeColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Project - +- BroadcastHashJoin - :- Filter - : +- ColumnarToRow - : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : +- ReusedSubquery - +- BroadcastExchange - +- CometNativeColumnarToRow - +- CometProject - +- CometFilter - +- CometNativeScan parquet spark_catalog.default.date_dim + +- CometExchange + +- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometProject + +- CometBroadcastHashJoin + :- CometFilter + : +- CometNativeScan parquet spark_catalog.default.store_sales + : +- SubqueryBroadcast + : +- BroadcastExchange + : +- CometNativeColumnarToRow + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.date_dim + +- CometBroadcastExchange + +- CometProject + +- CometFilter + +- CometNativeScan parquet spark_catalog.default.date_dim -Comet accelerated 23 out of 47 eligible operators (48%). Final plan contains 10 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 40 out of 51 eligible operators (78%). Final plan contains 5 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q53.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q53.native_datafusion/extended.txt index 5c7a26057d..936cde052f 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q53.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q53.native_datafusion/extended.txt @@ -4,39 +4,34 @@ TakeOrderedAndProject +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] +- CometNativeColumnarToRow +- CometSort - +- CometColumnarExchange - +- HashAggregate - +- CometNativeColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.item - : : +- BroadcastExchange - : : +- Filter - : : +- ColumnarToRow - : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- CometNativeColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.date_dim - +- BroadcastExchange - +- CometNativeColumnarToRow - +- CometFilter - +- CometNativeScan parquet spark_catalog.default.store + +- CometExchange + +- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.item + : : +- CometBroadcastExchange + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : +- SubqueryBroadcast + : : +- BroadcastExchange + : : +- CometNativeColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.date_dim + +- CometBroadcastExchange + +- CometFilter + +- CometNativeScan parquet spark_catalog.default.store -Comet accelerated 14 out of 33 eligible operators (42%). Final plan contains 7 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 27 out of 33 eligible operators (81%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q54.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q54.native_datafusion/extended.txt index 273efea475..e7fd40210f 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q54.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q54.native_datafusion/extended.txt @@ -1,116 +1,96 @@ -TakeOrderedAndProject -+- HashAggregate - +- CometNativeColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- HashAggregate - +- CometNativeColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- CometNativeColumnarToRow - : : : : +- CometHashAggregate - : : : : +- CometColumnarExchange - : : : : +- HashAggregate - : : : : +- Project - : : : : +- BroadcastHashJoin - : : : : :- Project - : : : : : +- BroadcastHashJoin - : : : : : :- Project - : : : : : : +- BroadcastHashJoin - : : : : : : :- Union - : : : : : : : :- Project - : : : : : : : : +- Filter - : : : : : : : : +- ColumnarToRow - : : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : : : : +- SubqueryBroadcast - : : : : : : : : +- BroadcastExchange - : : : : : : : : +- CometNativeColumnarToRow - : : : : : : : : +- CometProject - : : : : : : : : +- CometFilter - : : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : : : : +- Project - : : : : : : : +- Filter - : : : : : : : +- ColumnarToRow - : : : : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : : : +- ReusedSubquery - : : : : : : +- BroadcastExchange - : : : : : : +- CometNativeColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : : +- BroadcastExchange - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.customer - : : : +- BroadcastExchange - : : : +- Filter - : : : +- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : : :- Subquery - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometHashAggregate - : : : : : +- CometExchange - : : : : : +- CometHashAggregate - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : +- Subquery - : : : : +- CometNativeColumnarToRow - : : : : +- CometHashAggregate - : : : : +- CometExchange - : : : : +- CometHashAggregate - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.customer_address - : +- BroadcastExchange - : +- CometNativeColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.store - +- BroadcastExchange - +- CometNativeColumnarToRow - +- CometProject - +- CometFilter - : :- Subquery - : : +- CometNativeColumnarToRow - : : +- CometHashAggregate - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- Subquery - : +- CometNativeColumnarToRow - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.date_dim - +- CometNativeScan parquet spark_catalog.default.date_dim +CometNativeColumnarToRow ++- CometTakeOrderedAndProject + +- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometHashAggregate + : : : : +- CometExchange + : : : : +- CometHashAggregate + : : : : +- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometProject + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometProject + : : : : : : +- CometBroadcastHashJoin + : : : : : : :- CometUnion + : : : : : : : :- CometProject + : : : : : : : : +- CometFilter + : : : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : : : : : +- SubqueryBroadcast + : : : : : : : : +- BroadcastExchange + : : : : : : : : +- CometNativeColumnarToRow + : : : : : : : : +- CometProject + : : : : : : : : +- CometFilter + : : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : : +- CometProject + : : : : : : : +- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.web_sales + : : : : : : : +- SubqueryBroadcast + : : : : : : : +- BroadcastExchange + : : : : : : : +- CometNativeColumnarToRow + : : : : : : : +- CometProject + : : : : : : : +- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : +- CometBroadcastExchange + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : : : +- CometBroadcastExchange + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.customer + : : : +- CometBroadcastExchange + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometNativeColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : : :- Subquery + : : : : : +- CometNativeColumnarToRow + : : : : : +- CometHashAggregate + : : : : : +- CometExchange + : : : : : +- CometHashAggregate + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- Subquery + : : : : +- CometNativeColumnarToRow + : : : : +- CometHashAggregate + : : : : +- CometExchange + : : : : +- CometHashAggregate + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.customer_address + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.store + +- CometBroadcastExchange + +- CometProject + +- CometFilter + : :- ReusedSubquery + : +- ReusedSubquery + +- CometNativeScan parquet spark_catalog.default.date_dim -Comet accelerated 51 out of 96 eligible operators (53%). Final plan contains 18 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 78 out of 88 eligible operators (88%). Final plan contains 6 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q56.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q56.native_datafusion/extended.txt index ad055b48b6..d52c40d57b 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q56.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q56.native_datafusion/extended.txt @@ -1,115 +1,110 @@ -TakeOrderedAndProject -+- HashAggregate - +- CometNativeColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Union - :- HashAggregate - : +- CometNativeColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- Filter - : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.customer_address - : +- BroadcastExchange - : +- CometNativeColumnarToRow - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.item - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.item - :- HashAggregate - : +- CometNativeColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- Filter - : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- ReusedSubquery - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.customer_address - : +- BroadcastExchange - : +- CometNativeColumnarToRow - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.item - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.item - +- HashAggregate - +- CometNativeColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Filter - : : : +- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : +- ReusedSubquery - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- CometNativeColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.customer_address - +- BroadcastExchange - +- CometNativeColumnarToRow - +- CometProject - +- CometBroadcastHashJoin - :- CometFilter - : +- CometNativeScan parquet spark_catalog.default.item - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometNativeScan parquet spark_catalog.default.item +CometNativeColumnarToRow ++- CometTakeOrderedAndProject + +- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometUnion + :- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometNativeColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.customer_address + : +- CometBroadcastExchange + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.item + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.item + :- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometNativeColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.customer_address + : +- CometBroadcastExchange + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.item + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.item + +- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.web_sales + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometNativeColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.customer_address + +- CometBroadcastExchange + +- CometProject + +- CometBroadcastHashJoin + :- CometFilter + : +- CometNativeScan parquet spark_catalog.default.item + +- CometBroadcastExchange + +- CometProject + +- CometFilter + +- CometNativeScan parquet spark_catalog.default.item -Comet accelerated 49 out of 96 eligible operators (51%). Final plan contains 17 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 98 out of 104 eligible operators (94%). Final plan contains 4 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q57.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q57.native_datafusion/extended.txt index 66c510aaa0..024bfeeeea 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q57.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q57.native_datafusion/extended.txt @@ -10,111 +10,96 @@ TakeOrderedAndProject : : +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] : : +- CometNativeColumnarToRow : : +- CometSort - : : +- CometColumnarExchange - : : +- HashAggregate - : : +- CometNativeColumnarToRow - : : +- CometColumnarExchange - : : +- HashAggregate - : : +- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- Project - : : : : +- BroadcastHashJoin - : : : : :- CometNativeColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : +- BroadcastExchange - : : : : +- Filter - : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.call_center + : : +- CometExchange + : : +- CometHashAggregate + : : +- CometExchange + : : +- CometHashAggregate + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : : +- CometBroadcastExchange + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometNativeColumnarToRow + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.call_center : +- BroadcastExchange : +- Project : +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] : +- CometNativeColumnarToRow : +- CometSort - : +- CometColumnarExchange - : +- HashAggregate - : +- CometNativeColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : +- BroadcastExchange - : : : +- Filter - : : : +- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- CometNativeColumnarToRow - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.call_center + : +- CometExchange + : +- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : +- CometBroadcastExchange + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometNativeColumnarToRow + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.call_center +- BroadcastExchange +- Project +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] +- CometNativeColumnarToRow +- CometSort - +- CometColumnarExchange - +- HashAggregate - +- CometNativeColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.item - : : +- BroadcastExchange - : : +- Filter - : : +- ColumnarToRow - : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- CometNativeColumnarToRow - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.date_dim - +- BroadcastExchange - +- CometNativeColumnarToRow - +- CometFilter - +- CometNativeScan parquet spark_catalog.default.call_center + +- CometExchange + +- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.item + : : +- CometBroadcastExchange + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : +- SubqueryBroadcast + : : +- BroadcastExchange + : : +- CometNativeColumnarToRow + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.date_dim + +- CometBroadcastExchange + +- CometFilter + +- CometNativeScan parquet spark_catalog.default.call_center -Comet accelerated 36 out of 97 eligible operators (37%). Final plan contains 21 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 75 out of 97 eligible operators (77%). Final plan contains 6 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q58.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q58.native_datafusion/extended.txt index ff5ea85d39..da8f8ec939 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q58.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q58.native_datafusion/extended.txt @@ -1,123 +1,142 @@ -TakeOrderedAndProject -+- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- Filter - : : +- HashAggregate - : : +- CometNativeColumnarToRow - : : +- CometColumnarExchange - : : +- HashAggregate - : : +- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- Filter - : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : : +- Subquery - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.item - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : : +- Subquery - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- Filter - : +- HashAggregate - : +- CometNativeColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Filter - : : : +- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : +- ReusedSubquery - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.item - : +- BroadcastExchange - : +- CometNativeColumnarToRow - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : : +- Subquery - : : +- CometNativeColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- CometNativeScan parquet spark_catalog.default.date_dim - +- BroadcastExchange - +- Filter - +- HashAggregate - +- CometNativeColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- Filter - : : +- ColumnarToRow - : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : +- ReusedSubquery - : +- BroadcastExchange - : +- CometNativeColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.item - +- BroadcastExchange - +- CometNativeColumnarToRow - +- CometProject - +- CometBroadcastHashJoin - :- CometFilter - : +- CometNativeScan parquet spark_catalog.default.date_dim - +- CometBroadcastExchange - +- CometProject - +- CometFilter - : +- Subquery - : +- CometNativeColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.date_dim - +- CometNativeScan parquet spark_catalog.default.date_dim +CometNativeColumnarToRow ++- CometTakeOrderedAndProject + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometFilter + : : +- CometHashAggregate + : : +- CometExchange + : : +- CometHashAggregate + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometNativeColumnarToRow + : : : : +- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : : +- Subquery + : : : : : +- CometNativeColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.item + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : : +- Subquery + : : : +- CometNativeColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometFilter + : +- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometNativeColumnarToRow + : : : +- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : : +- Subquery + : : : : +- CometNativeColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.item + : +- CometBroadcastExchange + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : : +- Subquery + : : +- CometNativeColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometNativeScan parquet spark_catalog.default.date_dim + +- CometBroadcastExchange + +- CometFilter + +- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.web_sales + : : +- SubqueryBroadcast + : : +- BroadcastExchange + : : +- CometNativeColumnarToRow + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : : +- Subquery + : : : +- CometNativeColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.item + +- CometBroadcastExchange + +- CometProject + +- CometBroadcastHashJoin + :- CometFilter + : +- CometNativeScan parquet spark_catalog.default.date_dim + +- CometBroadcastExchange + +- CometProject + +- CometFilter + : +- Subquery + : +- CometNativeColumnarToRow + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.date_dim + +- CometNativeScan parquet spark_catalog.default.date_dim -Comet accelerated 56 out of 104 eligible operators (53%). Final plan contains 17 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 118 out of 130 eligible operators (90%). Final plan contains 10 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q6.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q6.native_datafusion/extended.txt index f2cdb50e03..1cb21cb650 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q6.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q6.native_datafusion/extended.txt @@ -1,68 +1,57 @@ -TakeOrderedAndProject -+- Filter - +- HashAggregate - +- CometNativeColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.customer_address - : : : +- CometBroadcastExchange - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.customer - : : +- BroadcastExchange - : : +- Filter - : : +- ColumnarToRow - : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometProject - : : +- CometFilter - : : : +- Subquery - : : : +- CometNativeColumnarToRow - : : : +- CometHashAggregate - : : : +- CometExchange - : : : +- CometHashAggregate - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- CometNativeColumnarToRow - : +- CometProject - : +- CometFilter - : : +- Subquery - : : +- CometNativeColumnarToRow - : : +- CometHashAggregate - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- CometNativeScan parquet spark_catalog.default.date_dim - +- BroadcastExchange - +- CometNativeColumnarToRow - +- CometProject - +- CometBroadcastHashJoin - :- CometFilter - : +- CometNativeScan parquet spark_catalog.default.item - +- CometBroadcastExchange - +- CometFilter - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometFilter - +- CometNativeScan parquet spark_catalog.default.item +CometNativeColumnarToRow ++- CometTakeOrderedAndProject + +- CometFilter + +- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.customer_address + : : : +- CometBroadcastExchange + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.customer + : : +- CometBroadcastExchange + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : +- SubqueryBroadcast + : : +- BroadcastExchange + : : +- CometNativeColumnarToRow + : : +- CometProject + : : +- CometFilter + : : : +- Subquery + : : : +- CometNativeColumnarToRow + : : : +- CometHashAggregate + : : : +- CometExchange + : : : +- CometHashAggregate + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : : +- ReusedSubquery + : +- CometNativeScan parquet spark_catalog.default.date_dim + +- CometBroadcastExchange + +- CometProject + +- CometBroadcastHashJoin + :- CometFilter + : +- CometNativeScan parquet spark_catalog.default.item + +- CometBroadcastExchange + +- CometFilter + +- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometProject + +- CometFilter + +- CometNativeScan parquet spark_catalog.default.item -Comet accelerated 39 out of 58 eligible operators (67%). Final plan contains 8 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 48 out of 52 eligible operators (92%). Final plan contains 3 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q60.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q60.native_datafusion/extended.txt index ad055b48b6..d52c40d57b 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q60.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q60.native_datafusion/extended.txt @@ -1,115 +1,110 @@ -TakeOrderedAndProject -+- HashAggregate - +- CometNativeColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Union - :- HashAggregate - : +- CometNativeColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- Filter - : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.customer_address - : +- BroadcastExchange - : +- CometNativeColumnarToRow - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.item - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.item - :- HashAggregate - : +- CometNativeColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- Filter - : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- ReusedSubquery - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.customer_address - : +- BroadcastExchange - : +- CometNativeColumnarToRow - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.item - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.item - +- HashAggregate - +- CometNativeColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Filter - : : : +- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : +- ReusedSubquery - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- CometNativeColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.customer_address - +- BroadcastExchange - +- CometNativeColumnarToRow - +- CometProject - +- CometBroadcastHashJoin - :- CometFilter - : +- CometNativeScan parquet spark_catalog.default.item - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometNativeScan parquet spark_catalog.default.item +CometNativeColumnarToRow ++- CometTakeOrderedAndProject + +- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometUnion + :- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometNativeColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.customer_address + : +- CometBroadcastExchange + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.item + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.item + :- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometNativeColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.customer_address + : +- CometBroadcastExchange + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.item + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.item + +- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.web_sales + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometNativeColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.customer_address + +- CometBroadcastExchange + +- CometProject + +- CometBroadcastHashJoin + :- CometFilter + : +- CometNativeScan parquet spark_catalog.default.item + +- CometBroadcastExchange + +- CometProject + +- CometFilter + +- CometNativeScan parquet spark_catalog.default.item -Comet accelerated 49 out of 96 eligible operators (51%). Final plan contains 17 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 98 out of 104 eligible operators (94%). Final plan contains 4 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q61.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q61.native_datafusion/extended.txt index 025280d4ca..ef8907c7d8 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q61.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q61.native_datafusion/extended.txt @@ -1,101 +1,93 @@ Project -+- BroadcastNestedLoopJoin - :- HashAggregate - : +- CometNativeColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- Project - : : : : +- BroadcastHashJoin - : : : : :- Project - : : : : : +- BroadcastHashJoin - : : : : : :- Project - : : : : : : +- BroadcastHashJoin - : : : : : : :- Filter - : : : : : : : +- ColumnarToRow - : : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : : : +- SubqueryBroadcast - : : : : : : : +- BroadcastExchange - : : : : : : : +- CometNativeColumnarToRow - : : : : : : : +- CometProject - : : : : : : : +- CometFilter - : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : : : +- BroadcastExchange - : : : : : : +- CometNativeColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.store - : : : : : +- BroadcastExchange - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.promotion - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.customer - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.customer_address - : +- BroadcastExchange - : +- CometNativeColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.item ++- BroadcastNestedLoopJoin [COMET: BroadcastNestedLoopJoin is not supported] + :- CometNativeColumnarToRow + : +- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometProject + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometProject + : : : : : : +- CometBroadcastHashJoin + : : : : : : :- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : : : : +- SubqueryBroadcast + : : : : : : : +- BroadcastExchange + : : : : : : : +- CometNativeColumnarToRow + : : : : : : : +- CometProject + : : : : : : : +- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : +- CometBroadcastExchange + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.store + : : : : : +- CometBroadcastExchange + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.promotion + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.customer + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.customer_address + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.item +- BroadcastExchange - +- HashAggregate - +- CometNativeColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- Project - : : : : +- BroadcastHashJoin - : : : : :- Filter - : : : : : +- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : +- ReusedSubquery - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.store - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.customer - : +- BroadcastExchange - : +- CometNativeColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.customer_address - +- BroadcastExchange - +- CometNativeColumnarToRow - +- CometProject - +- CometFilter - +- CometNativeScan parquet spark_catalog.default.item + +- CometNativeColumnarToRow + +- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometNativeColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.store + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.customer + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.customer_address + +- CometBroadcastExchange + +- CometProject + +- CometFilter + +- CometNativeScan parquet spark_catalog.default.item -Comet accelerated 36 out of 83 eligible operators (43%). Final plan contains 16 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 80 out of 87 eligible operators (91%). Final plan contains 4 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q63.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q63.native_datafusion/extended.txt index 5c7a26057d..936cde052f 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q63.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q63.native_datafusion/extended.txt @@ -4,39 +4,34 @@ TakeOrderedAndProject +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] +- CometNativeColumnarToRow +- CometSort - +- CometColumnarExchange - +- HashAggregate - +- CometNativeColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.item - : : +- BroadcastExchange - : : +- Filter - : : +- ColumnarToRow - : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- CometNativeColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.date_dim - +- BroadcastExchange - +- CometNativeColumnarToRow - +- CometFilter - +- CometNativeScan parquet spark_catalog.default.store + +- CometExchange + +- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.item + : : +- CometBroadcastExchange + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : +- SubqueryBroadcast + : : +- BroadcastExchange + : : +- CometNativeColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.date_dim + +- CometBroadcastExchange + +- CometFilter + +- CometNativeScan parquet spark_catalog.default.store -Comet accelerated 14 out of 33 eligible operators (42%). Final plan contains 7 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 27 out of 33 eligible operators (81%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q64.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q64.native_datafusion/extended.txt index 98db64e5b0..f1fe6fb3e3 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q64.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q64.native_datafusion/extended.txt @@ -40,22 +40,20 @@ CometNativeColumnarToRow : : : : : : : : : : : : : : : :- CometProject : : : : : : : : : : : : : : : : +- CometSortMergeJoin : : : : : : : : : : : : : : : : :- CometSort - : : : : : : : : : : : : : : : : : +- CometColumnarExchange - : : : : : : : : : : : : : : : : : +- Project - : : : : : : : : : : : : : : : : : +- BroadcastHashJoin - : : : : : : : : : : : : : : : : : :- BroadcastExchange - : : : : : : : : : : : : : : : : : : +- Filter - : : : : : : : : : : : : : : : : : : +- ColumnarToRow - : : : : : : : : : : : : : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : : : : : : : : : : : : : : +- SubqueryBroadcast - : : : : : : : : : : : : : : : : : : +- BroadcastExchange - : : : : : : : : : : : : : : : : : : +- CometNativeColumnarToRow - : : : : : : : : : : : : : : : : : : +- CometFilter - : : : : : : : : : : : : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : : : : : : : : : : : : : : +- CometNativeColumnarToRow - : : : : : : : : : : : : : : : : : +- CometProject - : : : : : : : : : : : : : : : : : +- CometFilter - : : : : : : : : : : : : : : : : : +- CometNativeScan parquet spark_catalog.default.store_returns + : : : : : : : : : : : : : : : : : +- CometExchange + : : : : : : : : : : : : : : : : : +- CometProject + : : : : : : : : : : : : : : : : : +- CometBroadcastHashJoin + : : : : : : : : : : : : : : : : : :- CometBroadcastExchange + : : : : : : : : : : : : : : : : : : +- CometFilter + : : : : : : : : : : : : : : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : : : : : : : : : : : : : : : +- SubqueryBroadcast + : : : : : : : : : : : : : : : : : : +- BroadcastExchange + : : : : : : : : : : : : : : : : : : +- CometNativeColumnarToRow + : : : : : : : : : : : : : : : : : : +- CometFilter + : : : : : : : : : : : : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : : : : : : : : : : : : +- CometProject + : : : : : : : : : : : : : : : : : +- CometFilter + : : : : : : : : : : : : : : : : : +- CometNativeScan parquet spark_catalog.default.store_returns : : : : : : : : : : : : : : : : +- CometSort : : : : : : : : : : : : : : : : +- CometProject : : : : : : : : : : : : : : : : +- CometFilter @@ -162,22 +160,20 @@ CometNativeColumnarToRow : : : : : : : : : : : : : : :- CometProject : : : : : : : : : : : : : : : +- CometSortMergeJoin : : : : : : : : : : : : : : : :- CometSort - : : : : : : : : : : : : : : : : +- CometColumnarExchange - : : : : : : : : : : : : : : : : +- Project - : : : : : : : : : : : : : : : : +- BroadcastHashJoin - : : : : : : : : : : : : : : : : :- BroadcastExchange - : : : : : : : : : : : : : : : : : +- Filter - : : : : : : : : : : : : : : : : : +- ColumnarToRow - : : : : : : : : : : : : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : : : : : : : : : : : : : +- SubqueryBroadcast - : : : : : : : : : : : : : : : : : +- BroadcastExchange - : : : : : : : : : : : : : : : : : +- CometNativeColumnarToRow - : : : : : : : : : : : : : : : : : +- CometFilter - : : : : : : : : : : : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : : : : : : : : : : : : : +- CometNativeColumnarToRow - : : : : : : : : : : : : : : : : +- CometProject - : : : : : : : : : : : : : : : : +- CometFilter - : : : : : : : : : : : : : : : : +- CometNativeScan parquet spark_catalog.default.store_returns + : : : : : : : : : : : : : : : : +- CometExchange + : : : : : : : : : : : : : : : : +- CometProject + : : : : : : : : : : : : : : : : +- CometBroadcastHashJoin + : : : : : : : : : : : : : : : : :- CometBroadcastExchange + : : : : : : : : : : : : : : : : : +- CometFilter + : : : : : : : : : : : : : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : : : : : : : : : : : : : : +- SubqueryBroadcast + : : : : : : : : : : : : : : : : : +- BroadcastExchange + : : : : : : : : : : : : : : : : : +- CometNativeColumnarToRow + : : : : : : : : : : : : : : : : : +- CometFilter + : : : : : : : : : : : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : : : : : : : : : : : +- CometProject + : : : : : : : : : : : : : : : : +- CometFilter + : : : : : : : : : : : : : : : : +- CometNativeScan parquet spark_catalog.default.store_returns : : : : : : : : : : : : : : : +- CometSort : : : : : : : : : : : : : : : +- CometProject : : : : : : : : : : : : : : : +- CometFilter @@ -248,4 +244,4 @@ CometNativeColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.item -Comet accelerated 228 out of 242 eligible operators (94%). Final plan contains 7 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 238 out of 242 eligible operators (98%). Final plan contains 3 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q65.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q65.native_datafusion/extended.txt index 73e3aa7046..167a3386c2 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q65.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q65.native_datafusion/extended.txt @@ -1,60 +1,57 @@ -TakeOrderedAndProject -+- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- CometNativeColumnarToRow - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.store - : : +- BroadcastExchange - : : +- Filter - : : +- HashAggregate - : : +- CometNativeColumnarToRow - : : +- CometColumnarExchange - : : +- HashAggregate - : : +- Project - : : +- BroadcastHashJoin - : : :- Filter - : : : +- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- CometNativeColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.item - +- BroadcastExchange - +- Filter - +- HashAggregate - +- CometNativeColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- HashAggregate - +- CometNativeColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Project - +- BroadcastHashJoin - :- Filter - : +- ColumnarToRow - : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : +- ReusedSubquery - +- BroadcastExchange - +- CometNativeColumnarToRow - +- CometProject - +- CometFilter - +- CometNativeScan parquet spark_catalog.default.date_dim +CometNativeColumnarToRow ++- CometTakeOrderedAndProject + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.store + : : +- CometBroadcastExchange + : : +- CometFilter + : : +- CometHashAggregate + : : +- CometExchange + : : +- CometHashAggregate + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometNativeColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.item + +- CometBroadcastExchange + +- CometFilter + +- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometProject + +- CometBroadcastHashJoin + :- CometFilter + : +- CometNativeScan parquet spark_catalog.default.store_sales + : +- SubqueryBroadcast + : +- BroadcastExchange + : +- CometNativeColumnarToRow + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.date_dim + +- CometBroadcastExchange + +- CometProject + +- CometFilter + +- CometNativeScan parquet spark_catalog.default.date_dim -Comet accelerated 17 out of 48 eligible operators (35%). Final plan contains 10 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 48 out of 52 eligible operators (92%). Final plan contains 3 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q66.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q66.native_datafusion/extended.txt index 3f3c6b8d79..a7423e3a59 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q66.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q66.native_datafusion/extended.txt @@ -1,82 +1,74 @@ -TakeOrderedAndProject -+- HashAggregate - +- CometNativeColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Union - :- HashAggregate - : +- CometNativeColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- Project - : : : : +- BroadcastHashJoin - : : : : :- Filter - : : : : : +- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.warehouse - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.time_dim - : +- BroadcastExchange - : +- CometNativeColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.ship_mode - +- HashAggregate - +- CometNativeColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- Filter - : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- ReusedSubquery - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.warehouse - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- CometNativeColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.time_dim - +- BroadcastExchange - +- CometNativeColumnarToRow - +- CometProject - +- CometFilter - +- CometNativeScan parquet spark_catalog.default.ship_mode +CometNativeColumnarToRow ++- CometTakeOrderedAndProject + +- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometUnion + :- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.web_sales + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometNativeColumnarToRow + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.warehouse + : : : +- CometBroadcastExchange + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.time_dim + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.ship_mode + +- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometNativeColumnarToRow + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.warehouse + : : +- CometBroadcastExchange + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.time_dim + +- CometBroadcastExchange + +- CometProject + +- CometFilter + +- CometNativeScan parquet spark_catalog.default.ship_mode -Comet accelerated 27 out of 66 eligible operators (40%). Final plan contains 14 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 65 out of 69 eligible operators (94%). Final plan contains 3 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q67.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q67.native_datafusion/extended.txt index da753c17c1..94c6cc711d 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q67.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q67.native_datafusion/extended.txt @@ -5,42 +5,38 @@ TakeOrderedAndProject +- CometNativeColumnarToRow +- CometSort +- CometColumnarExchange - +- WindowGroupLimit - +- Sort - +- HashAggregate - +- CometNativeColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Expand - +- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Filter - : : : +- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- CometNativeColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.store - +- BroadcastExchange - +- CometNativeColumnarToRow - +- CometProject - +- CometFilter - +- CometNativeScan parquet spark_catalog.default.item + +- WindowGroupLimit [COMET: WindowGroupLimit is not supported] + +- CometNativeColumnarToRow + +- CometSort + +- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometExpand + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometNativeColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.store + +- CometBroadcastExchange + +- CometProject + +- CometFilter + +- CometNativeScan parquet spark_catalog.default.item -Comet accelerated 15 out of 37 eligible operators (40%). Final plan contains 7 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 30 out of 37 eligible operators (81%). Final plan contains 3 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q68.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q68.native_datafusion/extended.txt index cebcd85144..dbdaaa275a 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q68.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q68.native_datafusion/extended.txt @@ -1,56 +1,49 @@ -TakeOrderedAndProject -+- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- HashAggregate - : : +- CometNativeColumnarToRow - : : +- CometColumnarExchange - : : +- HashAggregate - : : +- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- Project - : : : : +- BroadcastHashJoin - : : : : :- Project - : : : : : +- BroadcastHashJoin - : : : : : :- Filter - : : : : : : +- ColumnarToRow - : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometNativeColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : : +- BroadcastExchange - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.store - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.household_demographics - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.customer_address - : +- BroadcastExchange - : +- CometNativeColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.customer - +- BroadcastExchange - +- CometNativeColumnarToRow +CometNativeColumnarToRow ++- CometTakeOrderedAndProject + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometHashAggregate + : : +- CometExchange + : : +- CometHashAggregate + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometProject + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : : : +- SubqueryBroadcast + : : : : : : +- BroadcastExchange + : : : : : : +- CometNativeColumnarToRow + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- CometBroadcastExchange + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.store + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.household_demographics + : : +- CometBroadcastExchange + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.customer_address + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.customer + +- CometBroadcastExchange +- CometFilter +- CometNativeScan parquet spark_catalog.default.customer_address -Comet accelerated 20 out of 45 eligible operators (44%). Final plan contains 9 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 43 out of 45 eligible operators (95%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q69.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q69.native_datafusion/extended.txt index f48c67d6e1..3e0ba48add 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q69.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q69.native_datafusion/extended.txt @@ -9,46 +9,52 @@ TakeOrderedAndProject : +- BroadcastHashJoin : :- Project : : +- BroadcastHashJoin - : : :- BroadcastHashJoin - : : : :- BroadcastHashJoin - : : : : :- CometNativeColumnarToRow - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.customer - : : : : +- BroadcastExchange - : : : : +- Project - : : : : +- BroadcastHashJoin - : : : : :- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow + : : :- BroadcastHashJoin [COMET: BuildRight with LeftAnti is not supported] + : : : :- CometNativeColumnarToRow + : : : : +- CometBroadcastHashJoin + : : : : :- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.customer + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometNativeScan parquet spark_catalog.default.store_sales + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometNativeColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange : : : : +- CometProject : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- BroadcastExchange - : : : +- Project - : : : +- BroadcastHashJoin - : : : :- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- ReusedSubquery - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow + : : : +- CometNativeColumnarToRow + : : : +- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometNativeScan parquet spark_catalog.default.web_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometNativeColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- BroadcastExchange - : : +- Project - : : +- BroadcastHashJoin - : : :- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : +- ReusedSubquery - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow + : : +- CometNativeColumnarToRow + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometNativeColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange : : +- CometProject : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -63,4 +69,4 @@ TakeOrderedAndProject +- CometFilter +- CometNativeScan parquet spark_catalog.default.customer_demographics -Comet accelerated 21 out of 53 eligible operators (39%). Final plan contains 11 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 41 out of 61 eligible operators (67%). Final plan contains 9 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q7.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q7.native_datafusion/extended.txt index b692dfa9be..42206d1be5 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q7.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q7.native_datafusion/extended.txt @@ -1,44 +1,39 @@ -TakeOrderedAndProject -+- HashAggregate - +- CometNativeColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- Filter - : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.customer_demographics - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- CometNativeColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.item - +- BroadcastExchange - +- CometNativeColumnarToRow - +- CometProject - +- CometFilter - +- CometNativeScan parquet spark_catalog.default.promotion +CometNativeColumnarToRow ++- CometTakeOrderedAndProject + +- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometNativeColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.customer_demographics + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.item + +- CometBroadcastExchange + +- CometProject + +- CometFilter + +- CometNativeScan parquet spark_catalog.default.promotion -Comet accelerated 16 out of 35 eligible operators (45%). Final plan contains 7 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 33 out of 35 eligible operators (94%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q70.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q70.native_datafusion/extended.txt index f222353c17..befbf52ae5 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q70.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q70.native_datafusion/extended.txt @@ -11,19 +11,18 @@ TakeOrderedAndProject +- Expand +- Project +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- Filter - : : +- ColumnarToRow - : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- CometNativeColumnarToRow + :- CometNativeColumnarToRow + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : +- SubqueryBroadcast + : : +- BroadcastExchange + : : +- CometNativeColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange : +- CometProject : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -37,29 +36,31 @@ TakeOrderedAndProject +- Project +- Filter +- Window - +- WindowGroupLimit - +- Sort - +- HashAggregate - +- CometNativeColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- Filter - : : +- ColumnarToRow - : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : +- ReusedSubquery - : +- BroadcastExchange - : +- CometNativeColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.store - +- BroadcastExchange - +- CometNativeColumnarToRow - +- CometProject - +- CometFilter - +- CometNativeScan parquet spark_catalog.default.date_dim + +- WindowGroupLimit [COMET: WindowGroupLimit is not supported] + +- CometNativeColumnarToRow + +- CometSort + +- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : +- SubqueryBroadcast + : : +- BroadcastExchange + : : +- CometNativeColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.store + +- CometBroadcastExchange + +- CometProject + +- CometFilter + +- CometNativeScan parquet spark_catalog.default.date_dim -Comet accelerated 18 out of 53 eligible operators (33%). Final plan contains 10 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 37 out of 57 eligible operators (64%). Final plan contains 7 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q71.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q71.native_datafusion/extended.txt index 23844f41f3..7f76dd192d 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q71.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q71.native_datafusion/extended.txt @@ -1,62 +1,63 @@ CometNativeColumnarToRow +- CometSort - +- CometColumnarExchange - +- HashAggregate - +- CometNativeColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.item - : +- Union - : :- Project - : : +- BroadcastHashJoin - : : :- Filter - : : : +- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : :- Project - : : +- BroadcastHashJoin - : : :- Filter - : : : +- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : +- ReusedSubquery - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- Project - : +- BroadcastHashJoin - : :- Filter - : : +- ColumnarToRow - : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : +- ReusedSubquery - : +- BroadcastExchange - : +- CometNativeColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.date_dim - +- BroadcastExchange - +- CometNativeColumnarToRow - +- CometProject - +- CometFilter - +- CometNativeScan parquet spark_catalog.default.time_dim + +- CometExchange + +- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.item + : +- CometUnion + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.web_sales + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometNativeColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometNativeColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : +- SubqueryBroadcast + : : +- BroadcastExchange + : : +- CometNativeColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.date_dim + +- CometBroadcastExchange + +- CometProject + +- CometFilter + +- CometNativeScan parquet spark_catalog.default.time_dim -Comet accelerated 21 out of 49 eligible operators (42%). Final plan contains 11 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 51 out of 57 eligible operators (89%). Final plan contains 4 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q72.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q72.native_datafusion/extended.txt index 62c2b17b7e..623a68ddab 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q72.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q72.native_datafusion/extended.txt @@ -6,77 +6,67 @@ CometNativeColumnarToRow +- CometProject +- CometSortMergeJoin :- CometSort - : +- CometColumnarExchange - : +- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- Project - : : : : +- BroadcastHashJoin - : : : : :- Project - : : : : : +- BroadcastHashJoin - : : : : : :- Project - : : : : : : +- BroadcastHashJoin - : : : : : : :- Project - : : : : : : : +- BroadcastHashJoin - : : : : : : : :- Project - : : : : : : : : +- BroadcastHashJoin - : : : : : : : : :- Project - : : : : : : : : : +- BroadcastHashJoin - : : : : : : : : : :- Filter - : : : : : : : : : : +- ColumnarToRow - : : : : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : : : : : : +- SubqueryBroadcast - : : : : : : : : : : +- BroadcastExchange - : : : : : : : : : : +- CometNativeColumnarToRow - : : : : : : : : : : +- CometProject - : : : : : : : : : : +- CometFilter - : : : : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : : : : : : +- BroadcastExchange - : : : : : : : : : +- CometNativeColumnarToRow - : : : : : : : : : +- CometFilter - : : : : : : : : : +- CometNativeScan parquet spark_catalog.default.inventory - : : : : : : : : +- BroadcastExchange - : : : : : : : : +- CometNativeColumnarToRow - : : : : : : : : +- CometFilter - : : : : : : : : +- CometNativeScan parquet spark_catalog.default.warehouse - : : : : : : : +- BroadcastExchange - : : : : : : : +- CometNativeColumnarToRow - : : : : : : : +- CometFilter - : : : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : : : +- BroadcastExchange - : : : : : : +- CometNativeColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.customer_demographics - : : : : : +- BroadcastExchange - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.household_demographics - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- CometNativeColumnarToRow - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.promotion + : +- CometExchange + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometProject + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometProject + : : : : : : +- CometBroadcastHashJoin + : : : : : : :- CometProject + : : : : : : : +- CometBroadcastHashJoin + : : : : : : : :- CometProject + : : : : : : : : +- CometBroadcastHashJoin + : : : : : : : : :- CometProject + : : : : : : : : : +- CometBroadcastHashJoin + : : : : : : : : : :- CometFilter + : : : : : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : : : : : : : +- SubqueryBroadcast + : : : : : : : : : : +- BroadcastExchange + : : : : : : : : : : +- CometNativeColumnarToRow + : : : : : : : : : : +- CometProject + : : : : : : : : : : +- CometFilter + : : : : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : : : : +- CometBroadcastExchange + : : : : : : : : : +- CometFilter + : : : : : : : : : +- CometNativeScan parquet spark_catalog.default.inventory + : : : : : : : : +- CometBroadcastExchange + : : : : : : : : +- CometFilter + : : : : : : : : +- CometNativeScan parquet spark_catalog.default.warehouse + : : : : : : : +- CometBroadcastExchange + : : : : : : : +- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : : : : +- CometBroadcastExchange + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.customer_demographics + : : : : : +- CometBroadcastExchange + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.household_demographics + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.promotion +- CometSort +- CometExchange +- CometProject +- CometFilter +- CometNativeScan parquet spark_catalog.default.catalog_returns -Comet accelerated 37 out of 68 eligible operators (54%). Final plan contains 12 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 66 out of 68 eligible operators (97%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q73.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q73.native_datafusion/extended.txt index 40230582bb..1b61be2937 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q73.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q73.native_datafusion/extended.txt @@ -1,47 +1,41 @@ CometNativeColumnarToRow +- CometSort - +- CometColumnarExchange - +- Project - +- BroadcastHashJoin - :- Filter - : +- HashAggregate - : +- CometNativeColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- Filter - : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.store - : +- BroadcastExchange - : +- CometNativeColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.household_demographics - +- BroadcastExchange - +- CometNativeColumnarToRow - +- CometProject - +- CometFilter - +- CometNativeScan parquet spark_catalog.default.customer + +- CometExchange + +- CometProject + +- CometBroadcastHashJoin + :- CometFilter + : +- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometNativeColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.store + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.household_demographics + +- CometBroadcastExchange + +- CometProject + +- CometFilter + +- CometNativeScan parquet spark_catalog.default.customer -Comet accelerated 18 out of 37 eligible operators (48%). Final plan contains 8 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 35 out of 37 eligible operators (94%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q74.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q74.native_datafusion/extended.txt index 5d218998be..51a3f5ca94 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q74.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q74.native_datafusion/extended.txt @@ -1,105 +1,98 @@ -TakeOrderedAndProject -+- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- BroadcastHashJoin - : : :- Filter - : : : +- HashAggregate - : : : +- CometNativeColumnarToRow - : : : +- CometColumnarExchange - : : : +- HashAggregate - : : : +- Project - : : : +- BroadcastHashJoin - : : : :- Project - : : : : +- BroadcastHashJoin - : : : : :- CometNativeColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.customer - : : : : +- BroadcastExchange - : : : : +- Filter - : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- HashAggregate - : : +- CometNativeColumnarToRow - : : +- CometColumnarExchange - : : +- HashAggregate - : : +- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.customer - : : : +- BroadcastExchange - : : : +- Filter - : : : +- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- Filter - : +- HashAggregate - : +- CometNativeColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.customer - : : +- BroadcastExchange - : : +- Filter - : : +- ColumnarToRow - : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : +- ReusedSubquery - : +- BroadcastExchange - : +- CometNativeColumnarToRow - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.date_dim - +- BroadcastExchange - +- HashAggregate - +- CometNativeColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- CometNativeColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.customer - : +- BroadcastExchange - : +- Filter - : +- ColumnarToRow - : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : +- ReusedSubquery - +- BroadcastExchange - +- CometNativeColumnarToRow - +- CometFilter - +- CometNativeScan parquet spark_catalog.default.date_dim +CometNativeColumnarToRow ++- CometTakeOrderedAndProject + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometHashAggregate + : : : +- CometExchange + : : : +- CometHashAggregate + : : : +- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.customer + : : : : +- CometBroadcastExchange + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometNativeColumnarToRow + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometHashAggregate + : : +- CometExchange + : : +- CometHashAggregate + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.customer + : : : +- CometBroadcastExchange + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometNativeColumnarToRow + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometFilter + : +- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.customer + : : +- CometBroadcastExchange + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.web_sales + : : +- SubqueryBroadcast + : : +- BroadcastExchange + : : +- CometNativeColumnarToRow + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.date_dim + +- CometBroadcastExchange + +- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.customer + : +- CometBroadcastExchange + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.web_sales + : +- SubqueryBroadcast + : +- BroadcastExchange + : +- CometNativeColumnarToRow + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.date_dim + +- CometBroadcastExchange + +- CometFilter + +- CometNativeScan parquet spark_catalog.default.date_dim -Comet accelerated 28 out of 85 eligible operators (32%). Final plan contains 18 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 83 out of 91 eligible operators (91%). Final plan contains 5 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q75.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q75.native_datafusion/extended.txt index 42106a0ffd..a943308f1e 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q75.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q75.native_datafusion/extended.txt @@ -15,28 +15,25 @@ CometNativeColumnarToRow : :- CometProject : : +- CometSortMergeJoin : : :- CometSort - : : : +- CometColumnarExchange - : : : +- Project - : : : +- BroadcastHashJoin - : : : :- Project - : : : : +- BroadcastHashJoin - : : : : :- Filter - : : : : : +- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometExchange + : : : +- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometNativeColumnarToRow + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : +- CometBroadcastExchange + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- CometSort : : +- CometExchange : : +- CometProject @@ -45,24 +42,25 @@ CometNativeColumnarToRow : :- CometProject : : +- CometSortMergeJoin : : :- CometSort - : : : +- CometColumnarExchange - : : : +- Project - : : : +- BroadcastHashJoin - : : : :- Project - : : : : +- BroadcastHashJoin - : : : : :- Filter - : : : : : +- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : +- ReusedSubquery - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometExchange + : : : +- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometNativeColumnarToRow + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : +- CometBroadcastExchange + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- CometSort : : +- CometExchange : : +- CometProject @@ -71,24 +69,25 @@ CometNativeColumnarToRow : +- CometProject : +- CometSortMergeJoin : :- CometSort - : : +- CometColumnarExchange - : : +- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- Filter - : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- ReusedSubquery - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.item - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometExchange + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.web_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometNativeColumnarToRow + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.item + : : +- CometBroadcastExchange + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim : +- CometSort : +- CometExchange : +- CometProject @@ -107,28 +106,25 @@ CometNativeColumnarToRow :- CometProject : +- CometSortMergeJoin : :- CometSort - : : +- CometColumnarExchange - : : +- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- Filter - : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.item - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometExchange + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometNativeColumnarToRow + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.item + : : +- CometBroadcastExchange + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim : +- CometSort : +- CometExchange : +- CometProject @@ -137,24 +133,25 @@ CometNativeColumnarToRow :- CometProject : +- CometSortMergeJoin : :- CometSort - : : +- CometColumnarExchange - : : +- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- Filter - : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- ReusedSubquery - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.item - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometExchange + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometNativeColumnarToRow + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.item + : : +- CometBroadcastExchange + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim : +- CometSort : +- CometExchange : +- CometProject @@ -163,28 +160,29 @@ CometNativeColumnarToRow +- CometProject +- CometSortMergeJoin :- CometSort - : +- CometColumnarExchange - : +- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Filter - : : : +- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : +- ReusedSubquery - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.item - : +- BroadcastExchange - : +- CometNativeColumnarToRow - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometExchange + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.web_sales + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometNativeColumnarToRow + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.item + : +- CometBroadcastExchange + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.date_dim +- CometSort +- CometExchange +- CometProject +- CometFilter +- CometNativeScan parquet spark_catalog.default.web_returns -Comet accelerated 111 out of 167 eligible operators (66%). Final plan contains 21 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 167 out of 179 eligible operators (93%). Final plan contains 7 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q77.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q77.native_datafusion/extended.txt index 5d2ca1a9d0..835f7a449a 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q77.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q77.native_datafusion/extended.txt @@ -5,131 +5,138 @@ TakeOrderedAndProject +- HashAggregate +- Expand +- Union + :- CometNativeColumnarToRow + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometHashAggregate + : : +- CometExchange + : : +- CometHashAggregate + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometNativeColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.store + : +- CometBroadcastExchange + : +- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.store_returns + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometNativeColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.store :- Project - : +- BroadcastHashJoin - : :- HashAggregate - : : +- CometNativeColumnarToRow - : : +- CometColumnarExchange - : : +- HashAggregate - : : +- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- Filter - : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.store - : +- BroadcastExchange - : +- HashAggregate - : +- CometNativeColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Filter - : : : +- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.store_returns [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : +- ReusedSubquery - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- CometNativeColumnarToRow - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.store - :- Project - : +- BroadcastNestedLoopJoin + : +- BroadcastNestedLoopJoin [COMET: BroadcastNestedLoopJoin is not supported] : :- BroadcastExchange - : : +- HashAggregate - : : +- CometNativeColumnarToRow - : : +- CometColumnarExchange - : : +- HashAggregate - : : +- Project - : : +- BroadcastHashJoin - : : :- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : +- ReusedSubquery - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- HashAggregate - : +- CometNativeColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- ColumnarToRow - : : +- Scan parquet spark_catalog.default.catalog_returns [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : +- ReusedSubquery - : +- BroadcastExchange - : +- CometNativeColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.date_dim - +- Project - +- BroadcastHashJoin - :- HashAggregate - : +- CometNativeColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Filter - : : : +- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : +- ReusedSubquery - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- CometNativeColumnarToRow - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.web_page - +- BroadcastExchange - +- HashAggregate - +- CometNativeColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- Filter - : : +- ColumnarToRow - : : +- Scan parquet spark_catalog.default.web_returns [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : +- ReusedSubquery - : +- BroadcastExchange - : +- CometNativeColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.date_dim - +- BroadcastExchange - +- CometNativeColumnarToRow - +- CometFilter - +- CometNativeScan parquet spark_catalog.default.web_page + : : +- CometNativeColumnarToRow + : : +- CometHashAggregate + : : +- CometExchange + : : +- CometHashAggregate + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometNativeColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometNativeColumnarToRow + : +- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometNativeScan parquet spark_catalog.default.catalog_returns + : : +- SubqueryBroadcast + : : +- BroadcastExchange + : : +- CometNativeColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.date_dim + +- CometNativeColumnarToRow + +- CometProject + +- CometBroadcastHashJoin + :- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.web_sales + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometNativeColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.web_page + +- CometBroadcastExchange + +- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.web_returns + : : +- SubqueryBroadcast + : : +- BroadcastExchange + : : +- CometNativeColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.date_dim + +- CometBroadcastExchange + +- CometFilter + +- CometNativeScan parquet spark_catalog.default.web_page -Comet accelerated 36 out of 109 eligible operators (33%). Final plan contains 24 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 109 out of 129 eligible operators (84%). Final plan contains 11 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q78.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q78.native_datafusion/extended.txt index af4b7000b9..ebfdcc3c62 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q78.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q78.native_datafusion/extended.txt @@ -14,15 +14,14 @@ TakeOrderedAndProject : : : +- CometFilter : : : +- CometSortMergeJoin : : : :- CometSort - : : : : +- CometColumnarExchange - : : : : +- Filter - : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometExchange + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometNativeColumnarToRow + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- CometSort : : : +- CometExchange : : : +- CometProject @@ -42,11 +41,14 @@ TakeOrderedAndProject : : +- CometFilter : : +- CometSortMergeJoin : : :- CometSort - : : : +- CometColumnarExchange - : : : +- Filter - : : : +- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : +- ReusedSubquery + : : : +- CometExchange + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.web_sales + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometNativeColumnarToRow + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- CometSort : : +- CometExchange : : +- CometProject @@ -66,11 +68,14 @@ TakeOrderedAndProject : +- CometFilter : +- CometSortMergeJoin : :- CometSort - : : +- CometColumnarExchange - : : +- Filter - : : +- ColumnarToRow - : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : +- ReusedSubquery + : : +- CometExchange + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : +- SubqueryBroadcast + : : +- BroadcastExchange + : : +- CometNativeColumnarToRow + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim : +- CometSort : +- CometExchange : +- CometProject @@ -80,4 +85,4 @@ TakeOrderedAndProject +- CometFilter +- CometNativeScan parquet spark_catalog.default.date_dim -Comet accelerated 64 out of 76 eligible operators (84%). Final plan contains 5 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 74 out of 82 eligible operators (90%). Final plan contains 4 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q79.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q79.native_datafusion/extended.txt index f24d730d8b..526b86f475 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q79.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q79.native_datafusion/extended.txt @@ -1,44 +1,39 @@ -TakeOrderedAndProject -+- Project - +- BroadcastHashJoin - :- HashAggregate - : +- CometNativeColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- Filter - : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.store - : +- BroadcastExchange - : +- CometNativeColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.household_demographics - +- BroadcastExchange - +- CometNativeColumnarToRow +CometNativeColumnarToRow ++- CometTakeOrderedAndProject + +- CometProject + +- CometBroadcastHashJoin + :- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometNativeColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.store + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.household_demographics + +- CometBroadcastExchange +- CometProject +- CometFilter +- CometNativeScan parquet spark_catalog.default.customer -Comet accelerated 16 out of 35 eligible operators (45%). Final plan contains 7 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 33 out of 35 eligible operators (94%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q8.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q8.native_datafusion/extended.txt index 8c9eb7b8c9..cb04fc12c8 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q8.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q8.native_datafusion/extended.txt @@ -1,56 +1,52 @@ -TakeOrderedAndProject -+- HashAggregate - +- CometNativeColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Filter - : : : +- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- CometNativeColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.store - +- BroadcastExchange - +- CometNativeColumnarToRow - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometBroadcastHashJoin - :- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.customer_address - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.customer_address - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometNativeScan parquet spark_catalog.default.customer +CometNativeColumnarToRow ++- CometTakeOrderedAndProject + +- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometNativeColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.store + +- CometBroadcastExchange + +- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometBroadcastHashJoin + :- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.customer_address + +- CometBroadcastExchange + +- CometProject + +- CometFilter + +- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.customer_address + +- CometBroadcastExchange + +- CometProject + +- CometFilter + +- CometNativeScan parquet spark_catalog.default.customer -Comet accelerated 32 out of 48 eligible operators (66%). Final plan contains 6 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 46 out of 48 eligible operators (95%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q80.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q80.native_datafusion/extended.txt index 6853afccad..003be45ee5 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q80.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q80.native_datafusion/extended.txt @@ -19,16 +19,15 @@ CometNativeColumnarToRow : : : : :- CometProject : : : : : +- CometSortMergeJoin : : : : : :- CometSort - : : : : : : +- CometColumnarExchange - : : : : : : +- Filter - : : : : : : +- ColumnarToRow - : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometNativeColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : +- CometExchange + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : : : +- SubqueryBroadcast + : : : : : : +- BroadcastExchange + : : : : : : +- CometNativeColumnarToRow + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : : +- CometSort : : : : : +- CometExchange : : : : : +- CometProject @@ -64,11 +63,15 @@ CometNativeColumnarToRow : : : : :- CometProject : : : : : +- CometSortMergeJoin : : : : : :- CometSort - : : : : : : +- CometColumnarExchange - : : : : : : +- Filter - : : : : : : +- ColumnarToRow - : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : : +- ReusedSubquery + : : : : : : +- CometExchange + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : : : +- SubqueryBroadcast + : : : : : : +- BroadcastExchange + : : : : : : +- CometNativeColumnarToRow + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : : +- CometSort : : : : : +- CometExchange : : : : : +- CometProject @@ -104,11 +107,15 @@ CometNativeColumnarToRow : : : :- CometProject : : : : +- CometSortMergeJoin : : : : :- CometSort - : : : : : +- CometColumnarExchange - : : : : : +- Filter - : : : : : +- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : +- ReusedSubquery + : : : : : +- CometExchange + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.web_sales + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometNativeColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : +- CometSort : : : : +- CometExchange : : : : +- CometProject @@ -131,4 +138,4 @@ CometNativeColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.promotion -Comet accelerated 117 out of 127 eligible operators (92%). Final plan contains 5 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 129 out of 135 eligible operators (95%). Final plan contains 4 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q81.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q81.native_datafusion/extended.txt index 841b2ca52d..949d50a867 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q81.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q81.native_datafusion/extended.txt @@ -1,75 +1,70 @@ -TakeOrderedAndProject -+- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Filter - : : : +- HashAggregate - : : : +- CometNativeColumnarToRow - : : : +- CometColumnarExchange - : : : +- HashAggregate - : : : +- Project - : : : +- BroadcastHashJoin - : : : :- Project - : : : : +- BroadcastHashJoin - : : : : :- Filter - : : : : : +- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.catalog_returns [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.customer_address - : : +- BroadcastExchange - : : +- Filter - : : +- HashAggregate - : : +- CometNativeColumnarToRow - : : +- CometColumnarExchange - : : +- HashAggregate - : : +- HashAggregate - : : +- CometNativeColumnarToRow - : : +- CometColumnarExchange - : : +- HashAggregate - : : +- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- Filter - : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.catalog_returns [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- ReusedSubquery - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.customer_address - : +- BroadcastExchange - : +- CometNativeColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.customer - +- BroadcastExchange - +- CometNativeColumnarToRow +CometNativeColumnarToRow ++- CometTakeOrderedAndProject + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometHashAggregate + : : : +- CometExchange + : : : +- CometHashAggregate + : : : +- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_returns + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometNativeColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.customer_address + : : +- CometBroadcastExchange + : : +- CometFilter + : : +- CometHashAggregate + : : +- CometExchange + : : +- CometHashAggregate + : : +- CometHashAggregate + : : +- CometExchange + : : +- CometHashAggregate + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.catalog_returns + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometNativeColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.customer_address + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.customer + +- CometBroadcastExchange +- CometProject +- CometFilter +- CometNativeScan parquet spark_catalog.default.customer_address -Comet accelerated 24 out of 61 eligible operators (39%). Final plan contains 12 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 61 out of 65 eligible operators (93%). Final plan contains 3 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q82.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q82.native_datafusion/extended.txt index 8abf5c1fb1..30fca82ec8 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q82.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q82.native_datafusion/extended.txt @@ -1,38 +1,34 @@ CometNativeColumnarToRow +- CometTakeOrderedAndProject +- CometHashAggregate - +- CometColumnarExchange - +- HashAggregate - +- Project - +- BroadcastHashJoin - :- BroadcastExchange - : +- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.item - : : +- BroadcastExchange - : : +- Project - : : +- Filter - : : +- ColumnarToRow - : : +- Scan parquet spark_catalog.default.inventory [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- CometNativeColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.date_dim - +- CometNativeColumnarToRow - +- CometProject - +- CometFilter - +- CometNativeScan parquet spark_catalog.default.store_sales + +- CometExchange + +- CometHashAggregate + +- CometProject + +- CometBroadcastHashJoin + :- CometBroadcastExchange + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.item + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.inventory + : : +- SubqueryBroadcast + : : +- BroadcastExchange + : : +- CometNativeColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.date_dim + +- CometProject + +- CometFilter + +- CometNativeScan parquet spark_catalog.default.store_sales -Comet accelerated 15 out of 30 eligible operators (50%). Final plan contains 6 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 28 out of 30 eligible operators (93%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q83.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q83.native_datafusion/extended.txt index 7616ffd2b1..3a16430014 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q83.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q83.native_datafusion/extended.txt @@ -1,116 +1,133 @@ -TakeOrderedAndProject -+- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- HashAggregate - : : +- CometNativeColumnarToRow - : : +- CometColumnarExchange - : : +- HashAggregate - : : +- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- Filter - : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.store_returns [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometNativeScan parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.item - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometNativeScan parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- HashAggregate - : +- CometNativeColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Filter - : : : +- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.catalog_returns [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : +- ReusedSubquery - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.item - : +- BroadcastExchange - : +- CometNativeColumnarToRow - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometNativeScan parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.date_dim - +- BroadcastExchange - +- HashAggregate - +- CometNativeColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- Filter - : : +- ColumnarToRow - : : +- Scan parquet spark_catalog.default.web_returns [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : +- ReusedSubquery - : +- BroadcastExchange - : +- CometNativeColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.item - +- BroadcastExchange - +- CometNativeColumnarToRow - +- CometProject - +- CometBroadcastHashJoin - :- CometFilter - : +- CometNativeScan parquet spark_catalog.default.date_dim - +- CometBroadcastExchange - +- CometProject - +- CometBroadcastHashJoin - :- CometNativeScan parquet spark_catalog.default.date_dim - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometNativeScan parquet spark_catalog.default.date_dim +CometNativeColumnarToRow ++- CometTakeOrderedAndProject + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometHashAggregate + : : +- CometExchange + : : +- CometHashAggregate + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.store_returns + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometNativeColumnarToRow + : : : : +- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.item + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.catalog_returns + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometNativeColumnarToRow + : : : +- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.item + : +- CometBroadcastExchange + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.date_dim + +- CometBroadcastExchange + +- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.web_returns + : : +- SubqueryBroadcast + : : +- BroadcastExchange + : : +- CometNativeColumnarToRow + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.item + +- CometBroadcastExchange + +- CometProject + +- CometBroadcastHashJoin + :- CometFilter + : +- CometNativeScan parquet spark_catalog.default.date_dim + +- CometBroadcastExchange + +- CometProject + +- CometBroadcastHashJoin + :- CometNativeScan parquet spark_catalog.default.date_dim + +- CometBroadcastExchange + +- CometProject + +- CometFilter + +- CometNativeScan parquet spark_catalog.default.date_dim -Comet accelerated 60 out of 101 eligible operators (59%). Final plan contains 13 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 121 out of 127 eligible operators (95%). Final plan contains 4 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q85.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q85.native_datafusion/extended.txt index aed7b365be..06537a5eec 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q85.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q85.native_datafusion/extended.txt @@ -1,64 +1,56 @@ -TakeOrderedAndProject -+- HashAggregate - +- CometNativeColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- Project - : : : : +- BroadcastHashJoin - : : : : :- Project - : : : : : +- BroadcastHashJoin - : : : : : :- Project - : : : : : : +- BroadcastHashJoin - : : : : : : :- BroadcastExchange - : : : : : : : +- Filter - : : : : : : : +- ColumnarToRow - : : : : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : : : +- SubqueryBroadcast - : : : : : : : +- BroadcastExchange - : : : : : : : +- CometNativeColumnarToRow - : : : : : : : +- CometProject - : : : : : : : +- CometFilter - : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : : : +- CometNativeColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.web_returns - : : : : : +- BroadcastExchange - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.web_page - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.customer_demographics - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.customer_demographics - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.customer_address - : +- BroadcastExchange - : +- CometNativeColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.date_dim - +- BroadcastExchange - +- CometNativeColumnarToRow - +- CometProject - +- CometFilter - +- CometNativeScan parquet spark_catalog.default.reason +CometNativeColumnarToRow ++- CometTakeOrderedAndProject + +- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometProject + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometProject + : : : : : : +- CometBroadcastHashJoin + : : : : : : :- CometBroadcastExchange + : : : : : : : +- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.web_sales + : : : : : : : +- SubqueryBroadcast + : : : : : : : +- BroadcastExchange + : : : : : : : +- CometNativeColumnarToRow + : : : : : : : +- CometProject + : : : : : : : +- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.web_returns + : : : : : +- CometBroadcastExchange + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.web_page + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.customer_demographics + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.customer_demographics + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.customer_address + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.date_dim + +- CometBroadcastExchange + +- CometProject + +- CometFilter + +- CometNativeScan parquet spark_catalog.default.reason -Comet accelerated 24 out of 52 eligible operators (46%). Final plan contains 10 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 50 out of 52 eligible operators (96%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q86.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q86.native_datafusion/extended.txt index a60a3158a4..c9117e2ae0 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q86.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q86.native_datafusion/extended.txt @@ -3,34 +3,30 @@ TakeOrderedAndProject +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] +- CometNativeColumnarToRow +- CometSort - +- CometColumnarExchange - +- HashAggregate - +- CometNativeColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Expand - +- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- Filter - : : +- ColumnarToRow - : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- CometNativeColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.date_dim - +- BroadcastExchange - +- CometNativeColumnarToRow - +- CometProject - +- CometFilter - +- CometNativeScan parquet spark_catalog.default.item + +- CometExchange + +- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometExpand + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.web_sales + : : +- SubqueryBroadcast + : : +- BroadcastExchange + : : +- CometNativeColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.date_dim + +- CometBroadcastExchange + +- CometProject + +- CometFilter + +- CometNativeScan parquet spark_catalog.default.item -Comet accelerated 12 out of 28 eligible operators (42%). Final plan contains 6 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 23 out of 28 eligible operators (82%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q87.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q87.native_datafusion/extended.txt index 7a9e62d57c..6676ec7a4b 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q87.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q87.native_datafusion/extended.txt @@ -7,76 +7,77 @@ HashAggregate :- BroadcastHashJoin [COMET: BuildRight with LeftAnti is not supported] : :- CometNativeColumnarToRow : : +- CometHashAggregate - : : +- CometColumnarExchange - : : +- HashAggregate - : : +- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- Filter - : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.customer + : : +- CometExchange + : : +- CometHashAggregate + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometNativeColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.customer : +- BroadcastExchange : +- CometNativeColumnarToRow : +- CometHashAggregate - : +- CometColumnarExchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Filter - : : : +- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : +- ReusedSubquery - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- CometNativeColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.customer + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometNativeColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.customer +- BroadcastExchange +- CometNativeColumnarToRow +- CometHashAggregate - +- CometColumnarExchange - +- HashAggregate - +- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- Filter - : : +- ColumnarToRow - : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : +- ReusedSubquery - : +- BroadcastExchange - : +- CometNativeColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.date_dim - +- BroadcastExchange - +- CometNativeColumnarToRow - +- CometProject - +- CometFilter - +- CometNativeScan parquet spark_catalog.default.customer + +- CometExchange + +- CometHashAggregate + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.web_sales + : : +- SubqueryBroadcast + : : +- BroadcastExchange + : : +- CometNativeColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.date_dim + +- CometBroadcastExchange + +- CometProject + +- CometFilter + +- CometNativeScan parquet spark_catalog.default.customer -Comet accelerated 28 out of 66 eligible operators (42%). Final plan contains 14 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 61 out of 74 eligible operators (82%). Final plan contains 7 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q89.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q89.native_datafusion/extended.txt index 5c7a26057d..936cde052f 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q89.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q89.native_datafusion/extended.txt @@ -4,39 +4,34 @@ TakeOrderedAndProject +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] +- CometNativeColumnarToRow +- CometSort - +- CometColumnarExchange - +- HashAggregate - +- CometNativeColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.item - : : +- BroadcastExchange - : : +- Filter - : : +- ColumnarToRow - : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- CometNativeColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.date_dim - +- BroadcastExchange - +- CometNativeColumnarToRow - +- CometFilter - +- CometNativeScan parquet spark_catalog.default.store + +- CometExchange + +- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.item + : : +- CometBroadcastExchange + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : +- SubqueryBroadcast + : : +- BroadcastExchange + : : +- CometNativeColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.date_dim + +- CometBroadcastExchange + +- CometFilter + +- CometNativeScan parquet spark_catalog.default.store -Comet accelerated 14 out of 33 eligible operators (42%). Final plan contains 7 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 27 out of 33 eligible operators (81%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q91.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q91.native_datafusion/extended.txt index 6f982615ae..28bc6eab2f 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q91.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q91.native_datafusion/extended.txt @@ -1,59 +1,51 @@ CometNativeColumnarToRow +- CometSort - +- CometColumnarExchange - +- HashAggregate - +- CometNativeColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- Project - : : : : +- BroadcastHashJoin - : : : : :- Project - : : : : : +- BroadcastHashJoin - : : : : : :- CometNativeColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.call_center - : : : : : +- BroadcastExchange - : : : : : +- Filter - : : : : : +- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.catalog_returns [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.customer - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.customer_address - : +- BroadcastExchange - : +- CometNativeColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.customer_demographics - +- BroadcastExchange - +- CometNativeColumnarToRow - +- CometProject - +- CometFilter - +- CometNativeScan parquet spark_catalog.default.household_demographics + +- CometExchange + +- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometProject + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.call_center + : : : : : +- CometBroadcastExchange + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_returns + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometNativeColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.customer + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.customer_address + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.customer_demographics + +- CometBroadcastExchange + +- CometProject + +- CometFilter + +- CometNativeScan parquet spark_catalog.default.household_demographics -Comet accelerated 23 out of 47 eligible operators (48%). Final plan contains 10 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 45 out of 47 eligible operators (95%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q92.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q92.native_datafusion/extended.txt index b3ca85cdaf..6c5b57b3a1 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q92.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q92.native_datafusion/extended.txt @@ -1,48 +1,47 @@ -HashAggregate -+- CometNativeColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Filter - : : : +- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.item - : +- BroadcastExchange - : +- Filter - : +- HashAggregate - : +- CometNativeColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- Filter - : : +- ColumnarToRow - : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : +- ReusedSubquery - : +- BroadcastExchange - : +- CometNativeColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.date_dim - +- BroadcastExchange - +- CometNativeColumnarToRow - +- CometProject - +- CometFilter - +- CometNativeScan parquet spark_catalog.default.date_dim +CometNativeColumnarToRow ++- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.web_sales + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometNativeColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.item + : +- CometBroadcastExchange + : +- CometFilter + : +- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.web_sales + : : +- SubqueryBroadcast + : : +- BroadcastExchange + : : +- CometNativeColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.date_dim + +- CometBroadcastExchange + +- CometProject + +- CometFilter + +- CometNativeScan parquet spark_catalog.default.date_dim -Comet accelerated 14 out of 38 eligible operators (36%). Final plan contains 8 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 38 out of 42 eligible operators (90%). Final plan contains 3 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q97.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q97.native_datafusion/extended.txt index e3b777a05a..b63930a880 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q97.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q97.native_datafusion/extended.txt @@ -6,36 +6,37 @@ CometNativeColumnarToRow +- CometSortMergeJoin :- CometSort : +- CometHashAggregate - : +- CometColumnarExchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- ColumnarToRow - : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- CometNativeColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometNativeScan parquet spark_catalog.default.store_sales + : : +- SubqueryBroadcast + : : +- BroadcastExchange + : : +- CometNativeColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.date_dim +- CometSort +- CometHashAggregate - +- CometColumnarExchange - +- HashAggregate - +- Project - +- BroadcastHashJoin - :- ColumnarToRow - : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : +- ReusedSubquery - +- BroadcastExchange - +- CometNativeColumnarToRow - +- CometProject - +- CometFilter - +- CometNativeScan parquet spark_catalog.default.date_dim + +- CometExchange + +- CometHashAggregate + +- CometProject + +- CometBroadcastHashJoin + :- CometNativeScan parquet spark_catalog.default.catalog_sales + : +- SubqueryBroadcast + : +- BroadcastExchange + : +- CometNativeColumnarToRow + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.date_dim + +- CometBroadcastExchange + +- CometProject + +- CometFilter + +- CometNativeScan parquet spark_catalog.default.date_dim -Comet accelerated 20 out of 33 eligible operators (60%). Final plan contains 6 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 33 out of 37 eligible operators (89%). Final plan contains 3 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q98.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q98.native_datafusion/extended.txt index af8cf29205..6cc173560b 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q98.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q98.native_datafusion/extended.txt @@ -6,33 +6,29 @@ CometNativeColumnarToRow +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] +- CometNativeColumnarToRow +- CometSort - +- CometColumnarExchange - +- HashAggregate - +- CometNativeColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- Filter - : : +- ColumnarToRow - : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- CometNativeColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.item - +- BroadcastExchange - +- CometNativeColumnarToRow - +- CometProject - +- CometFilter - +- CometNativeScan parquet spark_catalog.default.date_dim + +- CometExchange + +- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : +- SubqueryBroadcast + : : +- BroadcastExchange + : : +- CometNativeColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.item + +- CometBroadcastExchange + +- CometProject + +- CometFilter + +- CometNativeScan parquet spark_catalog.default.date_dim -Comet accelerated 15 out of 29 eligible operators (51%). Final plan contains 7 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 25 out of 29 eligible operators (86%). Final plan contains 3 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q1.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q1.native_datafusion/extended.txt index 91b07a7538..935c103614 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q1.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q1.native_datafusion/extended.txt @@ -1,61 +1,58 @@ -TakeOrderedAndProject -+- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Filter - : : : +- HashAggregate - : : : +- CometNativeColumnarToRow - : : : +- CometColumnarExchange - : : : +- HashAggregate - : : : +- Project - : : : +- BroadcastHashJoin - : : : :- Filter - : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.store_returns [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- Filter - : : +- HashAggregate - : : +- CometNativeColumnarToRow - : : +- CometColumnarExchange - : : +- HashAggregate - : : +- HashAggregate - : : +- CometNativeColumnarToRow - : : +- CometColumnarExchange - : : +- HashAggregate - : : +- Project - : : +- BroadcastHashJoin - : : :- Filter - : : : +- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.store_returns [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : +- ReusedSubquery - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- CometNativeColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.store - +- BroadcastExchange - +- CometNativeColumnarToRow +CometNativeColumnarToRow ++- CometTakeOrderedAndProject + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometHashAggregate + : : : +- CometExchange + : : : +- CometHashAggregate + : : : +- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.store_returns + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometNativeColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometFilter + : : +- CometHashAggregate + : : +- CometExchange + : : +- CometHashAggregate + : : +- CometHashAggregate + : : +- CometExchange + : : +- CometHashAggregate + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.store_returns + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometNativeColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.store + +- CometBroadcastExchange +- CometProject +- CometFilter +- CometNativeScan parquet spark_catalog.default.customer -Comet accelerated 18 out of 49 eligible operators (36%). Final plan contains 10 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 49 out of 53 eligible operators (92%). Final plan contains 3 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q10.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q10.native_datafusion/extended.txt index 40a8ef4a6e..2785c3d0b1 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q10.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q10.native_datafusion/extended.txt @@ -10,46 +10,52 @@ TakeOrderedAndProject : :- Project : : +- Filter : : +- BroadcastHashJoin - : : :- BroadcastHashJoin - : : : :- BroadcastHashJoin - : : : : :- CometNativeColumnarToRow - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.customer - : : : : +- BroadcastExchange - : : : : +- Project - : : : : +- BroadcastHashJoin - : : : : :- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow + : : :- BroadcastHashJoin [COMET: Unsupported join type ExistenceJoin(exists#1)] + : : : :- CometNativeColumnarToRow + : : : : +- CometBroadcastHashJoin + : : : : :- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.customer + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometNativeScan parquet spark_catalog.default.store_sales + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometNativeColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange : : : : +- CometProject : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- BroadcastExchange - : : : +- Project - : : : +- BroadcastHashJoin - : : : :- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- ReusedSubquery - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow + : : : +- CometNativeColumnarToRow + : : : +- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometNativeScan parquet spark_catalog.default.web_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometNativeColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- BroadcastExchange - : : +- Project - : : +- BroadcastHashJoin - : : :- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : +- ReusedSubquery - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow + : : +- CometNativeColumnarToRow + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometNativeColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange : : +- CometProject : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -64,4 +70,4 @@ TakeOrderedAndProject +- CometFilter +- CometNativeScan parquet spark_catalog.default.customer_demographics -Comet accelerated 21 out of 54 eligible operators (38%). Final plan contains 11 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 41 out of 62 eligible operators (66%). Final plan contains 9 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q11.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q11.native_datafusion/extended.txt index 47aa9517cf..6b081dd35b 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q11.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q11.native_datafusion/extended.txt @@ -1,106 +1,99 @@ -TakeOrderedAndProject -+- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Filter - : : : +- HashAggregate - : : : +- CometNativeColumnarToRow - : : : +- CometColumnarExchange - : : : +- HashAggregate - : : : +- Project - : : : +- BroadcastHashJoin - : : : :- Project - : : : : +- BroadcastHashJoin - : : : : :- CometNativeColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.customer - : : : : +- BroadcastExchange - : : : : +- Filter - : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- HashAggregate - : : +- CometNativeColumnarToRow - : : +- CometColumnarExchange - : : +- HashAggregate - : : +- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.customer - : : : +- BroadcastExchange - : : : +- Filter - : : : +- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- Filter - : +- HashAggregate - : +- CometNativeColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.customer - : : +- BroadcastExchange - : : +- Filter - : : +- ColumnarToRow - : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : +- ReusedSubquery - : +- BroadcastExchange - : +- CometNativeColumnarToRow - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.date_dim - +- BroadcastExchange - +- HashAggregate - +- CometNativeColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- CometNativeColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.customer - : +- BroadcastExchange - : +- Filter - : +- ColumnarToRow - : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : +- ReusedSubquery - +- BroadcastExchange - +- CometNativeColumnarToRow - +- CometFilter - +- CometNativeScan parquet spark_catalog.default.date_dim +CometNativeColumnarToRow ++- CometTakeOrderedAndProject + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometHashAggregate + : : : +- CometExchange + : : : +- CometHashAggregate + : : : +- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.customer + : : : : +- CometBroadcastExchange + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometNativeColumnarToRow + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometHashAggregate + : : +- CometExchange + : : +- CometHashAggregate + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.customer + : : : +- CometBroadcastExchange + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometNativeColumnarToRow + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometFilter + : +- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.customer + : : +- CometBroadcastExchange + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.web_sales + : : +- SubqueryBroadcast + : : +- BroadcastExchange + : : +- CometNativeColumnarToRow + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.date_dim + +- CometBroadcastExchange + +- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.customer + : +- CometBroadcastExchange + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.web_sales + : +- SubqueryBroadcast + : +- BroadcastExchange + : +- CometNativeColumnarToRow + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.date_dim + +- CometBroadcastExchange + +- CometFilter + +- CometNativeScan parquet spark_catalog.default.date_dim -Comet accelerated 28 out of 86 eligible operators (32%). Final plan contains 18 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 84 out of 92 eligible operators (91%). Final plan contains 5 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q12.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q12.native_datafusion/extended.txt index 15648408fc..a8c2531a38 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q12.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q12.native_datafusion/extended.txt @@ -3,33 +3,29 @@ TakeOrderedAndProject +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] +- CometNativeColumnarToRow +- CometSort - +- CometColumnarExchange - +- HashAggregate - +- CometNativeColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- Filter - : : +- ColumnarToRow - : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- CometNativeColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.item - +- BroadcastExchange - +- CometNativeColumnarToRow - +- CometProject - +- CometFilter - +- CometNativeScan parquet spark_catalog.default.date_dim + +- CometExchange + +- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.web_sales + : : +- SubqueryBroadcast + : : +- BroadcastExchange + : : +- CometNativeColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.item + +- CometBroadcastExchange + +- CometProject + +- CometFilter + +- CometNativeScan parquet spark_catalog.default.date_dim -Comet accelerated 12 out of 27 eligible operators (44%). Final plan contains 6 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 22 out of 27 eligible operators (81%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q13.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q13.native_datafusion/extended.txt index 9ff51aab85..42ea1c4966 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q13.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q13.native_datafusion/extended.txt @@ -1,48 +1,42 @@ -HashAggregate -+- CometNativeColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- Project - : : : : +- BroadcastHashJoin - : : : : :- Filter - : : : : : +- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.store - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.customer_address - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- CometNativeColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.customer_demographics - +- BroadcastExchange - +- CometNativeColumnarToRow - +- CometFilter - +- CometNativeScan parquet spark_catalog.default.household_demographics +CometNativeColumnarToRow ++- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometNativeColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.store + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.customer_address + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.customer_demographics + +- CometBroadcastExchange + +- CometFilter + +- CometNativeScan parquet spark_catalog.default.household_demographics -Comet accelerated 17 out of 38 eligible operators (44%). Final plan contains 8 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 36 out of 38 eligible operators (94%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q14a.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q14a.native_datafusion/extended.txt index 223593e3d0..0dcd47f57f 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q14a.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q14a.native_datafusion/extended.txt @@ -1,553 +1,554 @@ -TakeOrderedAndProject -+- HashAggregate - +- CometNativeColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Expand - +- Union - :- Project - : +- Filter +CometNativeColumnarToRow ++- CometTakeOrderedAndProject + +- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometExpand + +- CometUnion + :- CometProject + : +- CometFilter : : +- Subquery - : : +- HashAggregate - : : +- CometNativeColumnarToRow - : : +- CometColumnarExchange - : : +- HashAggregate - : : +- Union - : : :- Project - : : : +- BroadcastHashJoin - : : : :- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- ReusedSubquery - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : :- Project - : : : +- BroadcastHashJoin - : : : :- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- ReusedSubquery - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- Project - : : +- BroadcastHashJoin - : : :- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : +- ReusedSubquery - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- HashAggregate - : +- CometNativeColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- BroadcastHashJoin - : : : :- Filter - : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- Project - : : : +- BroadcastHashJoin - : : : :- CometNativeColumnarToRow - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : +- BroadcastExchange - : : : +- BroadcastHashJoin - : : : :- CometNativeColumnarToRow - : : : : +- CometHashAggregate - : : : : +- CometColumnarExchange - : : : : +- HashAggregate - : : : : +- Project - : : : : +- BroadcastHashJoin - : : : : :- Project - : : : : : +- BroadcastHashJoin - : : : : : :- Filter - : : : : : : +- ColumnarToRow - : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometNativeColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : : +- BroadcastExchange - : : : : : +- BroadcastHashJoin - : : : : : :- CometNativeColumnarToRow - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : : +- BroadcastExchange - : : : : : +- Project - : : : : : +- BroadcastHashJoin - : : : : : :- Project - : : : : : : +- BroadcastHashJoin - : : : : : : :- Filter - : : : : : : : +- ColumnarToRow - : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : : : +- ReusedSubquery - : : : : : : +- BroadcastExchange - : : : : : : +- CometNativeColumnarToRow - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : : +- BroadcastExchange - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- Project - : : : +- BroadcastHashJoin - : : : :- Project - : : : : +- BroadcastHashJoin - : : : : :- Filter - : : : : : +- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : +- ReusedSubquery - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- BroadcastHashJoin - : : :- CometNativeColumnarToRow - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.item - : : +- BroadcastExchange - : : +- Project - : : +- BroadcastHashJoin - : : :- CometNativeColumnarToRow - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.item - : : +- BroadcastExchange - : : +- BroadcastHashJoin - : : :- CometNativeColumnarToRow - : : : +- CometHashAggregate - : : : +- CometColumnarExchange - : : : +- HashAggregate - : : : +- Project - : : : +- BroadcastHashJoin - : : : :- Project - : : : : +- BroadcastHashJoin - : : : : :- Filter - : : : : : +- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : +- BroadcastExchange - : : : : +- BroadcastHashJoin - : : : : :- CometNativeColumnarToRow - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : +- BroadcastExchange - : : : : +- Project - : : : : +- BroadcastHashJoin - : : : : :- Project - : : : : : +- BroadcastHashJoin - : : : : : :- Filter - : : : : : : +- ColumnarToRow - : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : : +- ReusedSubquery - : : : : : +- BroadcastExchange - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- Filter - : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- ReusedSubquery - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.item - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- CometNativeColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.date_dim - :- Project - : +- Filter + : : +- CometNativeColumnarToRow + : : +- CometHashAggregate + : : +- CometExchange + : : +- CometHashAggregate + : : +- CometUnion + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometNativeScan parquet spark_catalog.default.store_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometNativeColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometNativeColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometNativeScan parquet spark_catalog.default.web_sales + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometNativeColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometNativeColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : +- CometBroadcastExchange + : : : +- CometBroadcastHashJoin + : : : :- CometHashAggregate + : : : : +- CometExchange + : : : : +- CometHashAggregate + : : : : +- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometProject + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : : : +- SubqueryBroadcast + : : : : : : +- BroadcastExchange + : : : : : : +- CometNativeColumnarToRow + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- CometBroadcastExchange + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : : : +- CometBroadcastExchange + : : : : : +- CometProject + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometProject + : : : : : : +- CometBroadcastHashJoin + : : : : : : :- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : : : : +- SubqueryBroadcast + : : : : : : : +- BroadcastExchange + : : : : : : : +- CometNativeColumnarToRow + : : : : : : : +- CometProject + : : : : : : : +- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : +- CometBroadcastExchange + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : : : +- CometBroadcastExchange + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.web_sales + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometNativeColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.item + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.item + : : +- CometBroadcastExchange + : : +- CometBroadcastHashJoin + : : :- CometHashAggregate + : : : +- CometExchange + : : : +- CometHashAggregate + : : : +- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometNativeColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometBroadcastHashJoin + : : : : :- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometProject + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : : : +- SubqueryBroadcast + : : : : : : +- BroadcastExchange + : : : : : : +- CometNativeColumnarToRow + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- CometBroadcastExchange + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.web_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometNativeColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.item + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.date_dim + :- CometProject + : +- CometFilter : : +- ReusedSubquery - : +- HashAggregate - : +- CometNativeColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- BroadcastHashJoin - : : : :- Filter - : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- ReusedSubquery - : : : +- BroadcastExchange - : : : +- Project - : : : +- BroadcastHashJoin - : : : :- CometNativeColumnarToRow - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : +- BroadcastExchange - : : : +- BroadcastHashJoin - : : : :- CometNativeColumnarToRow - : : : : +- CometHashAggregate - : : : : +- CometColumnarExchange - : : : : +- HashAggregate - : : : : +- Project - : : : : +- BroadcastHashJoin - : : : : :- Project - : : : : : +- BroadcastHashJoin - : : : : : :- Filter - : : : : : : +- ColumnarToRow - : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometNativeColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : : +- BroadcastExchange - : : : : : +- BroadcastHashJoin - : : : : : :- CometNativeColumnarToRow - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : : +- BroadcastExchange - : : : : : +- Project - : : : : : +- BroadcastHashJoin - : : : : : :- Project - : : : : : : +- BroadcastHashJoin - : : : : : : :- Filter - : : : : : : : +- ColumnarToRow - : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : : : +- ReusedSubquery - : : : : : : +- BroadcastExchange - : : : : : : +- CometNativeColumnarToRow - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : : +- BroadcastExchange - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- Project - : : : +- BroadcastHashJoin - : : : :- Project - : : : : +- BroadcastHashJoin - : : : : :- Filter - : : : : : +- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : +- ReusedSubquery - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- BroadcastHashJoin - : : :- CometNativeColumnarToRow - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.item - : : +- BroadcastExchange - : : +- Project - : : +- BroadcastHashJoin - : : :- CometNativeColumnarToRow - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.item - : : +- BroadcastExchange - : : +- BroadcastHashJoin - : : :- CometNativeColumnarToRow - : : : +- CometHashAggregate - : : : +- CometColumnarExchange - : : : +- HashAggregate - : : : +- Project - : : : +- BroadcastHashJoin - : : : :- Project - : : : : +- BroadcastHashJoin - : : : : :- Filter - : : : : : +- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : +- BroadcastExchange - : : : : +- BroadcastHashJoin - : : : : :- CometNativeColumnarToRow - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : +- BroadcastExchange - : : : : +- Project - : : : : +- BroadcastHashJoin - : : : : :- Project - : : : : : +- BroadcastHashJoin - : : : : : :- Filter - : : : : : : +- ColumnarToRow - : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : : +- ReusedSubquery - : : : : : +- BroadcastExchange - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- Filter - : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- ReusedSubquery - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.item - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- CometNativeColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.date_dim - +- Project - +- Filter + : +- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometNativeColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : +- CometBroadcastExchange + : : : +- CometBroadcastHashJoin + : : : :- CometHashAggregate + : : : : +- CometExchange + : : : : +- CometHashAggregate + : : : : +- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometProject + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : : : +- SubqueryBroadcast + : : : : : : +- BroadcastExchange + : : : : : : +- CometNativeColumnarToRow + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- CometBroadcastExchange + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : : : +- CometBroadcastExchange + : : : : : +- CometProject + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometProject + : : : : : : +- CometBroadcastHashJoin + : : : : : : :- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : : : : +- SubqueryBroadcast + : : : : : : : +- BroadcastExchange + : : : : : : : +- CometNativeColumnarToRow + : : : : : : : +- CometProject + : : : : : : : +- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : +- CometBroadcastExchange + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : : : +- CometBroadcastExchange + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.web_sales + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometNativeColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.item + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.item + : : +- CometBroadcastExchange + : : +- CometBroadcastHashJoin + : : :- CometHashAggregate + : : : +- CometExchange + : : : +- CometHashAggregate + : : : +- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometNativeColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometBroadcastHashJoin + : : : : :- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometProject + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : : : +- SubqueryBroadcast + : : : : : : +- BroadcastExchange + : : : : : : +- CometNativeColumnarToRow + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- CometBroadcastExchange + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.web_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometNativeColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.item + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.date_dim + +- CometProject + +- CometFilter : +- ReusedSubquery - +- HashAggregate - +- CometNativeColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- BroadcastHashJoin - : : :- Filter - : : : +- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : +- ReusedSubquery - : : +- BroadcastExchange - : : +- Project - : : +- BroadcastHashJoin - : : :- CometNativeColumnarToRow - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.item - : : +- BroadcastExchange - : : +- BroadcastHashJoin - : : :- CometNativeColumnarToRow - : : : +- CometHashAggregate - : : : +- CometColumnarExchange - : : : +- HashAggregate - : : : +- Project - : : : +- BroadcastHashJoin - : : : :- Project - : : : : +- BroadcastHashJoin - : : : : :- Filter - : : : : : +- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : +- BroadcastExchange - : : : : +- BroadcastHashJoin - : : : : :- CometNativeColumnarToRow - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : +- BroadcastExchange - : : : : +- Project - : : : : +- BroadcastHashJoin - : : : : :- Project - : : : : : +- BroadcastHashJoin - : : : : : :- Filter - : : : : : : +- ColumnarToRow - : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : : +- ReusedSubquery - : : : : : +- BroadcastExchange - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- Filter - : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- ReusedSubquery - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.item - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- BroadcastHashJoin - : :- CometNativeColumnarToRow - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.item - : +- BroadcastExchange - : +- Project - : +- BroadcastHashJoin - : :- CometNativeColumnarToRow - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.item - : +- BroadcastExchange - : +- BroadcastHashJoin - : :- CometNativeColumnarToRow - : : +- CometHashAggregate - : : +- CometColumnarExchange - : : +- HashAggregate - : : +- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- Filter - : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- BroadcastHashJoin - : : : :- CometNativeColumnarToRow - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : +- BroadcastExchange - : : : +- Project - : : : +- BroadcastHashJoin - : : : :- Project - : : : : +- BroadcastHashJoin - : : : : :- Filter - : : : : : +- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : +- ReusedSubquery - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Filter - : : : +- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : +- ReusedSubquery - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.item - : +- BroadcastExchange - : +- CometNativeColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.date_dim - +- BroadcastExchange - +- CometNativeColumnarToRow - +- CometProject - +- CometFilter - +- CometNativeScan parquet spark_catalog.default.date_dim + +- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.web_sales + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometNativeColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.item + : : +- CometBroadcastExchange + : : +- CometBroadcastHashJoin + : : :- CometHashAggregate + : : : +- CometExchange + : : : +- CometHashAggregate + : : : +- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometNativeColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometBroadcastHashJoin + : : : : :- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometProject + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : : : +- SubqueryBroadcast + : : : : : : +- BroadcastExchange + : : : : : : +- CometNativeColumnarToRow + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- CometBroadcastExchange + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.web_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometNativeColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.item + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometBroadcastHashJoin + : :- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.item + : +- CometBroadcastExchange + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.item + : +- CometBroadcastExchange + : +- CometBroadcastHashJoin + : :- CometHashAggregate + : : +- CometExchange + : : +- CometHashAggregate + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometNativeColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometNativeColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.web_sales + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometNativeColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.item + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.date_dim + +- CometBroadcastExchange + +- CometProject + +- CometFilter + +- CometNativeScan parquet spark_catalog.default.date_dim -Comet accelerated 164 out of 458 eligible operators (35%). Final plan contains 93 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 475 out of 526 eligible operators (90%). Final plan contains 26 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q14b.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q14b.native_datafusion/extended.txt index 25e8321ad7..b48995c0a5 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q14b.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q14b.native_datafusion/extended.txt @@ -1,408 +1,414 @@ -TakeOrderedAndProject -+- BroadcastHashJoin - :- Filter - : : +- Subquery - : : +- HashAggregate - : : +- CometNativeColumnarToRow - : : +- CometColumnarExchange - : : +- HashAggregate - : : +- Union - : : :- Project - : : : +- BroadcastHashJoin - : : : :- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- ReusedSubquery - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : :- Project - : : : +- BroadcastHashJoin - : : : :- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- ReusedSubquery - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- Project - : : +- BroadcastHashJoin - : : :- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : +- ReusedSubquery - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- HashAggregate - : +- CometNativeColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- BroadcastHashJoin - : : : :- Filter - : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : : +- ReusedSubquery - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : +- Subquery - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- Project - : : : +- BroadcastHashJoin - : : : :- CometNativeColumnarToRow - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : +- BroadcastExchange - : : : +- BroadcastHashJoin - : : : :- CometNativeColumnarToRow - : : : : +- CometHashAggregate - : : : : +- CometColumnarExchange - : : : : +- HashAggregate - : : : : +- Project - : : : : +- BroadcastHashJoin - : : : : :- Project - : : : : : +- BroadcastHashJoin - : : : : : :- Filter - : : : : : : +- ColumnarToRow - : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometNativeColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : : +- BroadcastExchange - : : : : : +- BroadcastHashJoin - : : : : : :- CometNativeColumnarToRow - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : : +- BroadcastExchange - : : : : : +- Project - : : : : : +- BroadcastHashJoin - : : : : : :- Project - : : : : : : +- BroadcastHashJoin - : : : : : : :- Filter - : : : : : : : +- ColumnarToRow - : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : : : +- ReusedSubquery - : : : : : : +- BroadcastExchange - : : : : : : +- CometNativeColumnarToRow - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : : +- BroadcastExchange - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- Project - : : : +- BroadcastHashJoin - : : : :- Project - : : : : +- BroadcastHashJoin - : : : : :- Filter - : : : : : +- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : +- ReusedSubquery - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- BroadcastHashJoin - : : :- CometNativeColumnarToRow - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.item - : : +- BroadcastExchange - : : +- Project - : : +- BroadcastHashJoin - : : :- CometNativeColumnarToRow - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.item - : : +- BroadcastExchange - : : +- BroadcastHashJoin - : : :- CometNativeColumnarToRow - : : : +- CometHashAggregate - : : : +- CometColumnarExchange - : : : +- HashAggregate - : : : +- Project - : : : +- BroadcastHashJoin - : : : :- Project - : : : : +- BroadcastHashJoin - : : : : :- Filter - : : : : : +- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : +- BroadcastExchange - : : : : +- BroadcastHashJoin - : : : : :- CometNativeColumnarToRow - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : +- BroadcastExchange - : : : : +- Project - : : : : +- BroadcastHashJoin - : : : : :- Project - : : : : : +- BroadcastHashJoin - : : : : : :- Filter - : : : : : : +- ColumnarToRow - : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : : +- ReusedSubquery - : : : : : +- BroadcastExchange - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- Filter - : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- ReusedSubquery - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.item - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- CometNativeColumnarToRow - : +- CometProject - : +- CometFilter - : : +- ReusedSubquery - : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- Subquery - : +- CometNativeColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.date_dim - +- BroadcastExchange - +- Filter - : +- ReusedSubquery - +- HashAggregate - +- CometNativeColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- BroadcastHashJoin - : : :- Filter - : : : +- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : : +- ReusedSubquery - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- Subquery - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- Project - : : +- BroadcastHashJoin - : : :- CometNativeColumnarToRow - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.item - : : +- BroadcastExchange - : : +- BroadcastHashJoin - : : :- CometNativeColumnarToRow - : : : +- CometHashAggregate - : : : +- CometColumnarExchange - : : : +- HashAggregate - : : : +- Project - : : : +- BroadcastHashJoin - : : : :- Project - : : : : +- BroadcastHashJoin - : : : : :- Filter - : : : : : +- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : +- BroadcastExchange - : : : : +- BroadcastHashJoin - : : : : :- CometNativeColumnarToRow - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : +- BroadcastExchange - : : : : +- Project - : : : : +- BroadcastHashJoin - : : : : :- Project - : : : : : +- BroadcastHashJoin - : : : : : :- Filter - : : : : : : +- ColumnarToRow - : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : : +- ReusedSubquery - : : : : : +- BroadcastExchange - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- Filter - : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- ReusedSubquery - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.item - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- BroadcastHashJoin - : :- CometNativeColumnarToRow - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.item - : +- BroadcastExchange - : +- Project - : +- BroadcastHashJoin - : :- CometNativeColumnarToRow - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.item - : +- BroadcastExchange - : +- BroadcastHashJoin - : :- CometNativeColumnarToRow - : : +- CometHashAggregate - : : +- CometColumnarExchange - : : +- HashAggregate - : : +- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- Filter - : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- BroadcastHashJoin - : : : :- CometNativeColumnarToRow - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : +- BroadcastExchange - : : : +- Project - : : : +- BroadcastHashJoin - : : : :- Project - : : : : +- BroadcastHashJoin - : : : : :- Filter - : : : : : +- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : +- ReusedSubquery - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Filter - : : : +- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : +- ReusedSubquery - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.item - : +- BroadcastExchange - : +- CometNativeColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.date_dim - +- BroadcastExchange - +- CometNativeColumnarToRow - +- CometProject - +- CometFilter - : +- ReusedSubquery - +- CometNativeScan parquet spark_catalog.default.date_dim - +- Subquery - +- CometNativeColumnarToRow - +- CometProject - +- CometFilter - +- CometNativeScan parquet spark_catalog.default.date_dim +CometNativeColumnarToRow ++- CometTakeOrderedAndProject + +- CometBroadcastHashJoin + :- CometFilter + : : +- Subquery + : : +- CometNativeColumnarToRow + : : +- CometHashAggregate + : : +- CometExchange + : : +- CometHashAggregate + : : +- CometUnion + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometNativeScan parquet spark_catalog.default.store_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometNativeColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometNativeColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometNativeScan parquet spark_catalog.default.web_sales + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometNativeColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometNativeColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : : +- Subquery + : : : : : +- CometNativeColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- Subquery + : : : : +- CometNativeColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : +- CometBroadcastExchange + : : : +- CometBroadcastHashJoin + : : : :- CometHashAggregate + : : : : +- CometExchange + : : : : +- CometHashAggregate + : : : : +- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometProject + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : : : +- SubqueryBroadcast + : : : : : : +- BroadcastExchange + : : : : : : +- CometNativeColumnarToRow + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- CometBroadcastExchange + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : : : +- CometBroadcastExchange + : : : : : +- CometProject + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometProject + : : : : : : +- CometBroadcastHashJoin + : : : : : : :- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : : : : +- SubqueryBroadcast + : : : : : : : +- BroadcastExchange + : : : : : : : +- CometNativeColumnarToRow + : : : : : : : +- CometProject + : : : : : : : +- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : +- CometBroadcastExchange + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : : : +- CometBroadcastExchange + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.web_sales + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometNativeColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.item + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.item + : : +- CometBroadcastExchange + : : +- CometBroadcastHashJoin + : : :- CometHashAggregate + : : : +- CometExchange + : : : +- CometHashAggregate + : : : +- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometNativeColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometBroadcastHashJoin + : : : : :- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometProject + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : : : +- SubqueryBroadcast + : : : : : : +- BroadcastExchange + : : : : : : +- CometNativeColumnarToRow + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- CometBroadcastExchange + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.web_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometNativeColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.item + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : : +- ReusedSubquery + : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- Subquery + : +- CometNativeColumnarToRow + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.date_dim + +- CometBroadcastExchange + +- CometFilter + : +- ReusedSubquery + +- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometNativeColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : : +- Subquery + : : : : +- CometNativeColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- Subquery + : : : +- CometNativeColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.item + : : +- CometBroadcastExchange + : : +- CometBroadcastHashJoin + : : :- CometHashAggregate + : : : +- CometExchange + : : : +- CometHashAggregate + : : : +- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometNativeColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometBroadcastHashJoin + : : : : :- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometProject + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : : : +- SubqueryBroadcast + : : : : : : +- BroadcastExchange + : : : : : : +- CometNativeColumnarToRow + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- CometBroadcastExchange + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.web_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometNativeColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.item + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometBroadcastHashJoin + : :- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.item + : +- CometBroadcastExchange + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.item + : +- CometBroadcastExchange + : +- CometBroadcastHashJoin + : :- CometHashAggregate + : : +- CometExchange + : : +- CometHashAggregate + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometNativeColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometNativeColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.web_sales + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometNativeColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.item + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.date_dim + +- CometBroadcastExchange + +- CometProject + +- CometFilter + : +- ReusedSubquery + +- CometNativeScan parquet spark_catalog.default.date_dim + +- Subquery + +- CometNativeColumnarToRow + +- CometProject + +- CometFilter + +- CometNativeScan parquet spark_catalog.default.date_dim -Comet accelerated 128 out of 337 eligible operators (37%). Final plan contains 69 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 343 out of 387 eligible operators (88%). Final plan contains 25 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q15.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q15.native_datafusion/extended.txt index d0d170db53..42432bdc12 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q15.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q15.native_datafusion/extended.txt @@ -1,36 +1,32 @@ -TakeOrderedAndProject -+- HashAggregate - +- CometNativeColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Filter - : : : +- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.customer - : +- BroadcastExchange - : +- CometNativeColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.customer_address - +- BroadcastExchange - +- CometNativeColumnarToRow - +- CometProject - +- CometFilter - +- CometNativeScan parquet spark_catalog.default.date_dim +CometNativeColumnarToRow ++- CometTakeOrderedAndProject + +- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometNativeColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.customer + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.customer_address + +- CometBroadcastExchange + +- CometProject + +- CometFilter + +- CometNativeScan parquet spark_catalog.default.date_dim -Comet accelerated 12 out of 28 eligible operators (42%). Final plan contains 6 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 26 out of 28 eligible operators (92%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q17.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q17.native_datafusion/extended.txt index 15c6bec5dd..f58c012650 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q17.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q17.native_datafusion/extended.txt @@ -1,70 +1,67 @@ -TakeOrderedAndProject -+- HashAggregate - +- CometNativeColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- Project - : : : : +- BroadcastHashJoin - : : : : :- Project - : : : : : +- BroadcastHashJoin - : : : : : :- Project - : : : : : : +- BroadcastHashJoin - : : : : : : :- Filter - : : : : : : : +- ColumnarToRow - : : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : : : +- SubqueryBroadcast - : : : : : : : +- BroadcastExchange - : : : : : : : +- CometNativeColumnarToRow - : : : : : : : +- CometProject - : : : : : : : +- CometFilter - : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : : : +- BroadcastExchange - : : : : : : +- Filter - : : : : : : +- ColumnarToRow - : : : : : : +- Scan parquet spark_catalog.default.store_returns [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometNativeColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : : +- BroadcastExchange - : : : : : +- Filter - : : : : : +- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : +- ReusedSubquery - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- CometNativeColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.store - +- BroadcastExchange - +- CometNativeColumnarToRow - +- CometProject - +- CometFilter - +- CometNativeScan parquet spark_catalog.default.item +CometNativeColumnarToRow ++- CometTakeOrderedAndProject + +- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometProject + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometProject + : : : : : : +- CometBroadcastHashJoin + : : : : : : :- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : : : : +- SubqueryBroadcast + : : : : : : : +- BroadcastExchange + : : : : : : : +- CometNativeColumnarToRow + : : : : : : : +- CometProject + : : : : : : : +- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : +- CometBroadcastExchange + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.store_returns + : : : : : : +- SubqueryBroadcast + : : : : : : +- BroadcastExchange + : : : : : : +- CometNativeColumnarToRow + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- CometBroadcastExchange + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometNativeColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.store + +- CometBroadcastExchange + +- CometProject + +- CometFilter + +- CometNativeScan parquet spark_catalog.default.item -Comet accelerated 22 out of 57 eligible operators (38%). Final plan contains 11 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 55 out of 61 eligible operators (90%). Final plan contains 4 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q18.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q18.native_datafusion/extended.txt index 53fa7b6342..b3485a3421 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q18.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q18.native_datafusion/extended.txt @@ -1,58 +1,51 @@ -TakeOrderedAndProject -+- HashAggregate - +- CometNativeColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Expand - +- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- Project - : : : : +- BroadcastHashJoin - : : : : :- Project - : : : : : +- BroadcastHashJoin - : : : : : :- Filter - : : : : : : +- ColumnarToRow - : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometNativeColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : : +- BroadcastExchange - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.customer_demographics - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.customer - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.customer_demographics - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.customer_address - : +- BroadcastExchange - : +- CometNativeColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.date_dim - +- BroadcastExchange - +- CometNativeColumnarToRow - +- CometProject - +- CometFilter - +- CometNativeScan parquet spark_catalog.default.item +CometNativeColumnarToRow ++- CometTakeOrderedAndProject + +- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometExpand + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometProject + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : : : +- SubqueryBroadcast + : : : : : : +- BroadcastExchange + : : : : : : +- CometNativeColumnarToRow + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- CometBroadcastExchange + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.customer_demographics + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.customer + : : : +- CometBroadcastExchange + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.customer_demographics + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.customer_address + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.date_dim + +- CometBroadcastExchange + +- CometProject + +- CometFilter + +- CometNativeScan parquet spark_catalog.default.item -Comet accelerated 21 out of 47 eligible operators (44%). Final plan contains 9 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 45 out of 47 eligible operators (95%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q20.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q20.native_datafusion/extended.txt index 6634edcbed..c4f54f4289 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q20.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q20.native_datafusion/extended.txt @@ -3,33 +3,29 @@ TakeOrderedAndProject +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] +- CometNativeColumnarToRow +- CometSort - +- CometColumnarExchange - +- HashAggregate - +- CometNativeColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- Filter - : : +- ColumnarToRow - : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- CometNativeColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.item - +- BroadcastExchange - +- CometNativeColumnarToRow - +- CometProject - +- CometFilter - +- CometNativeScan parquet spark_catalog.default.date_dim + +- CometExchange + +- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : +- SubqueryBroadcast + : : +- BroadcastExchange + : : +- CometNativeColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.item + +- CometBroadcastExchange + +- CometProject + +- CometFilter + +- CometNativeScan parquet spark_catalog.default.date_dim -Comet accelerated 12 out of 27 eligible operators (44%). Final plan contains 6 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 22 out of 27 eligible operators (81%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q21.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q21.native_datafusion/extended.txt index 407648ea84..8522eb4ab9 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q21.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q21.native_datafusion/extended.txt @@ -1,35 +1,31 @@ -TakeOrderedAndProject -+- Filter - +- HashAggregate - +- CometNativeColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Filter - : : : +- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.inventory [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.warehouse - : +- BroadcastExchange - : +- CometNativeColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.item - +- BroadcastExchange - +- CometNativeColumnarToRow - +- CometFilter - +- CometNativeScan parquet spark_catalog.default.date_dim +CometNativeColumnarToRow ++- CometTakeOrderedAndProject + +- CometFilter + +- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.inventory + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometNativeColumnarToRow + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.warehouse + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.item + +- CometBroadcastExchange + +- CometFilter + +- CometNativeScan parquet spark_catalog.default.date_dim -Comet accelerated 10 out of 27 eligible operators (37%). Final plan contains 6 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 25 out of 27 eligible operators (92%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q22.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q22.native_datafusion/extended.txt index e1766e64d8..6b78d21629 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q22.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q22.native_datafusion/extended.txt @@ -1,37 +1,33 @@ -TakeOrderedAndProject -+- HashAggregate - +- CometNativeColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Expand - +- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Filter - : : : +- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.inventory [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- CometNativeColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.item - +- BroadcastExchange - +- CometNativeColumnarToRow - +- CometFilter - +- CometNativeScan parquet spark_catalog.default.warehouse +CometNativeColumnarToRow ++- CometTakeOrderedAndProject + +- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometExpand + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.inventory + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometNativeColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.item + +- CometBroadcastExchange + +- CometFilter + +- CometNativeScan parquet spark_catalog.default.warehouse -Comet accelerated 12 out of 29 eligible operators (41%). Final plan contains 6 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 27 out of 29 eligible operators (93%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q23a.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q23a.native_datafusion/extended.txt index 7040e78da1..8a9990d536 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q23a.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q23a.native_datafusion/extended.txt @@ -8,80 +8,71 @@ CometNativeColumnarToRow : :- CometProject : : +- CometSortMergeJoin : : :- CometSort - : : : +- CometColumnarExchange - : : : +- Project - : : : +- BroadcastHashJoin - : : : :- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- Project - : : : +- Filter - : : : +- HashAggregate - : : : +- CometNativeColumnarToRow - : : : +- CometColumnarExchange - : : : +- HashAggregate - : : : +- Project - : : : +- BroadcastHashJoin - : : : :- Project - : : : : +- BroadcastHashJoin - : : : : :- Filter - : : : : : +- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.item + : : : +- CometExchange + : : : +- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometNativeColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometHashAggregate + : : : +- CometExchange + : : : +- CometHashAggregate + : : : +- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometNativeColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.item : : +- CometSort : : +- CometProject : : +- CometFilter : : : +- Subquery - : : : +- HashAggregate - : : : +- CometNativeColumnarToRow - : : : +- CometColumnarExchange - : : : +- HashAggregate - : : : +- HashAggregate - : : : +- CometNativeColumnarToRow - : : : +- CometColumnarExchange - : : : +- HashAggregate - : : : +- Project - : : : +- BroadcastHashJoin - : : : :- Project - : : : : +- BroadcastHashJoin - : : : : :- Filter - : : : : : +- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.customer - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometNativeColumnarToRow + : : : +- CometHashAggregate + : : : +- CometExchange + : : : +- CometHashAggregate + : : : +- CometHashAggregate + : : : +- CometExchange + : : : +- CometHashAggregate + : : : +- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometNativeColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.customer + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- CometHashAggregate : : +- CometExchange : : +- CometHashAggregate @@ -102,41 +93,41 @@ CometNativeColumnarToRow :- CometProject : +- CometSortMergeJoin : :- CometSort - : : +- CometColumnarExchange - : : +- Project - : : +- BroadcastHashJoin - : : :- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : +- ReusedSubquery - : : +- BroadcastExchange - : : +- Project - : : +- Filter - : : +- HashAggregate - : : +- CometNativeColumnarToRow - : : +- CometColumnarExchange - : : +- HashAggregate - : : +- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- Filter - : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.item + : : +- CometExchange + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometNativeScan parquet spark_catalog.default.web_sales + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometNativeColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometHashAggregate + : : +- CometExchange + : : +- CometHashAggregate + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometNativeColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.item : +- CometSort : +- CometProject : +- CometFilter @@ -157,4 +148,4 @@ CometNativeColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.date_dim -Comet accelerated 83 out of 138 eligible operators (60%). Final plan contains 20 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 130 out of 142 eligible operators (91%). Final plan contains 7 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q23b.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q23b.native_datafusion/extended.txt index 188775e7df..52090c668e 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q23b.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q23b.native_datafusion/extended.txt @@ -10,81 +10,72 @@ CometNativeColumnarToRow : : +- CometBroadcastHashJoin : : :- CometSortMergeJoin : : : :- CometSort - : : : : +- CometColumnarExchange - : : : : +- Project - : : : : +- BroadcastHashJoin - : : : : :- Filter - : : : : : +- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : +- BroadcastExchange - : : : : +- Project - : : : : +- Filter - : : : : +- HashAggregate - : : : : +- CometNativeColumnarToRow - : : : : +- CometColumnarExchange - : : : : +- HashAggregate - : : : : +- Project - : : : : +- BroadcastHashJoin - : : : : :- Project - : : : : : +- BroadcastHashJoin - : : : : : :- Filter - : : : : : : +- ColumnarToRow - : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometNativeColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : : +- BroadcastExchange - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : : +- CometExchange + : : : : +- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometNativeColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometHashAggregate + : : : : +- CometExchange + : : : : +- CometHashAggregate + : : : : +- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometProject + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : : : +- SubqueryBroadcast + : : : : : : +- BroadcastExchange + : : : : : : +- CometNativeColumnarToRow + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- CometBroadcastExchange + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.item : : : +- CometSort : : : +- CometProject : : : +- CometFilter : : : : +- Subquery - : : : : +- HashAggregate - : : : : +- CometNativeColumnarToRow - : : : : +- CometColumnarExchange - : : : : +- HashAggregate - : : : : +- HashAggregate - : : : : +- CometNativeColumnarToRow - : : : : +- CometColumnarExchange - : : : : +- HashAggregate - : : : : +- Project - : : : : +- BroadcastHashJoin - : : : : :- Project - : : : : : +- BroadcastHashJoin - : : : : : :- Filter - : : : : : : +- ColumnarToRow - : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometNativeColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : : +- BroadcastExchange - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.customer - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometNativeColumnarToRow + : : : : +- CometHashAggregate + : : : : +- CometExchange + : : : : +- CometHashAggregate + : : : : +- CometHashAggregate + : : : : +- CometExchange + : : : : +- CometHashAggregate + : : : : +- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometProject + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : : : +- SubqueryBroadcast + : : : : : : +- BroadcastExchange + : : : : : : +- CometNativeColumnarToRow + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- CometBroadcastExchange + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.customer + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- CometHashAggregate : : : +- CometExchange : : : +- CometHashAggregate @@ -131,42 +122,42 @@ CometNativeColumnarToRow : +- CometBroadcastHashJoin : :- CometSortMergeJoin : : :- CometSort - : : : +- CometColumnarExchange - : : : +- Project - : : : +- BroadcastHashJoin - : : : :- Filter - : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- ReusedSubquery - : : : +- BroadcastExchange - : : : +- Project - : : : +- Filter - : : : +- HashAggregate - : : : +- CometNativeColumnarToRow - : : : +- CometColumnarExchange - : : : +- HashAggregate - : : : +- Project - : : : +- BroadcastHashJoin - : : : :- Project - : : : : +- BroadcastHashJoin - : : : : :- Filter - : : : : : +- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.item + : : : +- CometExchange + : : : +- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.web_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometNativeColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometHashAggregate + : : : +- CometExchange + : : : +- CometHashAggregate + : : : +- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometNativeColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.item : : +- CometSort : : +- CometProject : : +- CometFilter @@ -209,4 +200,4 @@ CometNativeColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.date_dim -Comet accelerated 131 out of 190 eligible operators (68%). Final plan contains 20 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 180 out of 194 eligible operators (92%). Final plan contains 7 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q25.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q25.native_datafusion/extended.txt index 15c6bec5dd..f58c012650 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q25.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q25.native_datafusion/extended.txt @@ -1,70 +1,67 @@ -TakeOrderedAndProject -+- HashAggregate - +- CometNativeColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- Project - : : : : +- BroadcastHashJoin - : : : : :- Project - : : : : : +- BroadcastHashJoin - : : : : : :- Project - : : : : : : +- BroadcastHashJoin - : : : : : : :- Filter - : : : : : : : +- ColumnarToRow - : : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : : : +- SubqueryBroadcast - : : : : : : : +- BroadcastExchange - : : : : : : : +- CometNativeColumnarToRow - : : : : : : : +- CometProject - : : : : : : : +- CometFilter - : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : : : +- BroadcastExchange - : : : : : : +- Filter - : : : : : : +- ColumnarToRow - : : : : : : +- Scan parquet spark_catalog.default.store_returns [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometNativeColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : : +- BroadcastExchange - : : : : : +- Filter - : : : : : +- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : +- ReusedSubquery - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- CometNativeColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.store - +- BroadcastExchange - +- CometNativeColumnarToRow - +- CometProject - +- CometFilter - +- CometNativeScan parquet spark_catalog.default.item +CometNativeColumnarToRow ++- CometTakeOrderedAndProject + +- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometProject + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometProject + : : : : : : +- CometBroadcastHashJoin + : : : : : : :- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : : : : +- SubqueryBroadcast + : : : : : : : +- BroadcastExchange + : : : : : : : +- CometNativeColumnarToRow + : : : : : : : +- CometProject + : : : : : : : +- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : +- CometBroadcastExchange + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.store_returns + : : : : : : +- SubqueryBroadcast + : : : : : : +- BroadcastExchange + : : : : : : +- CometNativeColumnarToRow + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- CometBroadcastExchange + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometNativeColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.store + +- CometBroadcastExchange + +- CometProject + +- CometFilter + +- CometNativeScan parquet spark_catalog.default.item -Comet accelerated 22 out of 57 eligible operators (38%). Final plan contains 11 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 55 out of 61 eligible operators (90%). Final plan contains 4 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q26.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q26.native_datafusion/extended.txt index ceba8f5828..71fd3bbe6f 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q26.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q26.native_datafusion/extended.txt @@ -1,44 +1,39 @@ -TakeOrderedAndProject -+- HashAggregate - +- CometNativeColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- Filter - : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.customer_demographics - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- CometNativeColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.item - +- BroadcastExchange - +- CometNativeColumnarToRow - +- CometProject - +- CometFilter - +- CometNativeScan parquet spark_catalog.default.promotion +CometNativeColumnarToRow ++- CometTakeOrderedAndProject + +- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometNativeColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.customer_demographics + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.item + +- CometBroadcastExchange + +- CometProject + +- CometFilter + +- CometNativeScan parquet spark_catalog.default.promotion -Comet accelerated 16 out of 35 eligible operators (45%). Final plan contains 7 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 33 out of 35 eligible operators (94%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q27.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q27.native_datafusion/extended.txt index 20f7517d3e..15ab28f784 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q27.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q27.native_datafusion/extended.txt @@ -1,45 +1,40 @@ -TakeOrderedAndProject -+- HashAggregate - +- CometNativeColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Expand - +- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- Filter - : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.customer_demographics - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- CometNativeColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.store - +- BroadcastExchange - +- CometNativeColumnarToRow - +- CometProject - +- CometFilter - +- CometNativeScan parquet spark_catalog.default.item +CometNativeColumnarToRow ++- CometTakeOrderedAndProject + +- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometExpand + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometNativeColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.customer_demographics + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.store + +- CometBroadcastExchange + +- CometProject + +- CometFilter + +- CometNativeScan parquet spark_catalog.default.item -Comet accelerated 16 out of 36 eligible operators (44%). Final plan contains 7 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 34 out of 36 eligible operators (94%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q29.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q29.native_datafusion/extended.txt index bd9fa52aa7..f58c012650 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q29.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q29.native_datafusion/extended.txt @@ -1,75 +1,67 @@ -TakeOrderedAndProject -+- HashAggregate - +- CometNativeColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- Project - : : : : +- BroadcastHashJoin - : : : : :- Project - : : : : : +- BroadcastHashJoin - : : : : : :- Project - : : : : : : +- BroadcastHashJoin - : : : : : : :- Filter - : : : : : : : +- ColumnarToRow - : : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : : : +- SubqueryBroadcast - : : : : : : : +- BroadcastExchange - : : : : : : : +- CometNativeColumnarToRow - : : : : : : : +- CometProject - : : : : : : : +- CometFilter - : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : : : +- BroadcastExchange - : : : : : : +- Filter - : : : : : : +- ColumnarToRow - : : : : : : +- Scan parquet spark_catalog.default.store_returns [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometNativeColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : : +- BroadcastExchange - : : : : : +- Filter - : : : : : +- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- CometNativeColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.store - +- BroadcastExchange - +- CometNativeColumnarToRow - +- CometProject - +- CometFilter - +- CometNativeScan parquet spark_catalog.default.item +CometNativeColumnarToRow ++- CometTakeOrderedAndProject + +- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometProject + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometProject + : : : : : : +- CometBroadcastHashJoin + : : : : : : :- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : : : : +- SubqueryBroadcast + : : : : : : : +- BroadcastExchange + : : : : : : : +- CometNativeColumnarToRow + : : : : : : : +- CometProject + : : : : : : : +- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : +- CometBroadcastExchange + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.store_returns + : : : : : : +- SubqueryBroadcast + : : : : : : +- BroadcastExchange + : : : : : : +- CometNativeColumnarToRow + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- CometBroadcastExchange + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometNativeColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.store + +- CometBroadcastExchange + +- CometProject + +- CometFilter + +- CometNativeScan parquet spark_catalog.default.item -Comet accelerated 25 out of 61 eligible operators (40%). Final plan contains 12 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 55 out of 61 eligible operators (90%). Final plan contains 4 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q30.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q30.native_datafusion/extended.txt index c2e96eb412..18d0a12a03 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q30.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q30.native_datafusion/extended.txt @@ -1,75 +1,70 @@ -TakeOrderedAndProject -+- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Filter - : : : +- HashAggregate - : : : +- CometNativeColumnarToRow - : : : +- CometColumnarExchange - : : : +- HashAggregate - : : : +- Project - : : : +- BroadcastHashJoin - : : : :- Project - : : : : +- BroadcastHashJoin - : : : : :- Filter - : : : : : +- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.web_returns [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.customer_address - : : +- BroadcastExchange - : : +- Filter - : : +- HashAggregate - : : +- CometNativeColumnarToRow - : : +- CometColumnarExchange - : : +- HashAggregate - : : +- HashAggregate - : : +- CometNativeColumnarToRow - : : +- CometColumnarExchange - : : +- HashAggregate - : : +- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- Filter - : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.web_returns [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- ReusedSubquery - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.customer_address - : +- BroadcastExchange - : +- CometNativeColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.customer - +- BroadcastExchange - +- CometNativeColumnarToRow +CometNativeColumnarToRow ++- CometTakeOrderedAndProject + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometHashAggregate + : : : +- CometExchange + : : : +- CometHashAggregate + : : : +- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.web_returns + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometNativeColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.customer_address + : : +- CometBroadcastExchange + : : +- CometFilter + : : +- CometHashAggregate + : : +- CometExchange + : : +- CometHashAggregate + : : +- CometHashAggregate + : : +- CometExchange + : : +- CometHashAggregate + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.web_returns + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometNativeColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.customer_address + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.customer + +- CometBroadcastExchange +- CometProject +- CometFilter +- CometNativeScan parquet spark_catalog.default.customer_address -Comet accelerated 24 out of 61 eligible operators (39%). Final plan contains 12 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 61 out of 65 eligible operators (93%). Final plan contains 3 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q31.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q31.native_datafusion/extended.txt index 03bcf0413e..d747a5243e 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q31.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q31.native_datafusion/extended.txt @@ -1,150 +1,138 @@ CometNativeColumnarToRow +- CometSort - +- CometColumnarExchange - +- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- BroadcastHashJoin - : : : : :- HashAggregate - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometColumnarExchange - : : : : : +- HashAggregate - : : : : : +- Project - : : : : : +- BroadcastHashJoin - : : : : : :- Project - : : : : : : +- BroadcastHashJoin - : : : : : : :- Filter - : : : : : : : +- ColumnarToRow - : : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : : : +- SubqueryBroadcast - : : : : : : : +- BroadcastExchange - : : : : : : : +- CometNativeColumnarToRow - : : : : : : : +- CometFilter - : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : : : +- BroadcastExchange - : : : : : : +- CometNativeColumnarToRow - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : : +- BroadcastExchange - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.customer_address - : : : : +- BroadcastExchange - : : : : +- HashAggregate - : : : : +- CometNativeColumnarToRow - : : : : +- CometColumnarExchange - : : : : +- HashAggregate - : : : : +- Project - : : : : +- BroadcastHashJoin - : : : : :- Project - : : : : : +- BroadcastHashJoin - : : : : : :- Filter - : : : : : : +- ColumnarToRow - : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometNativeColumnarToRow - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : : +- BroadcastExchange - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.customer_address - : : : +- BroadcastExchange - : : : +- HashAggregate - : : : +- CometNativeColumnarToRow - : : : +- CometColumnarExchange - : : : +- HashAggregate - : : : +- Project - : : : +- BroadcastHashJoin - : : : :- Project - : : : : +- BroadcastHashJoin - : : : : :- Filter - : : : : : +- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.customer_address - : : +- BroadcastExchange - : : +- HashAggregate - : : +- CometNativeColumnarToRow - : : +- CometColumnarExchange - : : +- HashAggregate - : : +- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- Filter - : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- ReusedSubquery - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.customer_address - : +- BroadcastExchange - : +- HashAggregate - : +- CometNativeColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Filter - : : : +- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : +- ReusedSubquery - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- CometNativeColumnarToRow - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.customer_address - +- BroadcastExchange - +- HashAggregate - +- CometNativeColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- Filter - : : +- ColumnarToRow - : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : +- ReusedSubquery - : +- BroadcastExchange - : +- CometNativeColumnarToRow - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.date_dim - +- BroadcastExchange - +- CometNativeColumnarToRow - +- CometFilter - +- CometNativeScan parquet spark_catalog.default.customer_address + +- CometExchange + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometBroadcastHashJoin + : : : : :- CometHashAggregate + : : : : : +- CometExchange + : : : : : +- CometHashAggregate + : : : : : +- CometProject + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometProject + : : : : : : +- CometBroadcastHashJoin + : : : : : : :- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : : : : +- SubqueryBroadcast + : : : : : : : +- BroadcastExchange + : : : : : : : +- CometNativeColumnarToRow + : : : : : : : +- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : +- CometBroadcastExchange + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- CometBroadcastExchange + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.customer_address + : : : : +- CometBroadcastExchange + : : : : +- CometHashAggregate + : : : : +- CometExchange + : : : : +- CometHashAggregate + : : : : +- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometProject + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : : : +- SubqueryBroadcast + : : : : : : +- BroadcastExchange + : : : : : : +- CometNativeColumnarToRow + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- CometBroadcastExchange + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.customer_address + : : : +- CometBroadcastExchange + : : : +- CometHashAggregate + : : : +- CometExchange + : : : +- CometHashAggregate + : : : +- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometNativeColumnarToRow + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.customer_address + : : +- CometBroadcastExchange + : : +- CometHashAggregate + : : +- CometExchange + : : +- CometHashAggregate + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.web_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometNativeColumnarToRow + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.customer_address + : +- CometBroadcastExchange + : +- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.web_sales + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometNativeColumnarToRow + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.customer_address + +- CometBroadcastExchange + +- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.web_sales + : : +- SubqueryBroadcast + : : +- BroadcastExchange + : : +- CometNativeColumnarToRow + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.date_dim + +- CometBroadcastExchange + +- CometFilter + +- CometNativeScan parquet spark_catalog.default.customer_address -Comet accelerated 38 out of 120 eligible operators (31%). Final plan contains 28 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 117 out of 129 eligible operators (90%). Final plan contains 7 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q32.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q32.native_datafusion/extended.txt index f7096f5c3b..5d009dea2d 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q32.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q32.native_datafusion/extended.txt @@ -1,48 +1,47 @@ -HashAggregate -+- CometNativeColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Filter - : : : +- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.item - : +- BroadcastExchange - : +- Filter - : +- HashAggregate - : +- CometNativeColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- Filter - : : +- ColumnarToRow - : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : +- ReusedSubquery - : +- BroadcastExchange - : +- CometNativeColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.date_dim - +- BroadcastExchange - +- CometNativeColumnarToRow - +- CometProject - +- CometFilter - +- CometNativeScan parquet spark_catalog.default.date_dim +CometNativeColumnarToRow ++- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometNativeColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.item + : +- CometBroadcastExchange + : +- CometFilter + : +- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : +- SubqueryBroadcast + : : +- BroadcastExchange + : : +- CometNativeColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.date_dim + +- CometBroadcastExchange + +- CometProject + +- CometFilter + +- CometNativeScan parquet spark_catalog.default.date_dim -Comet accelerated 14 out of 38 eligible operators (36%). Final plan contains 8 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 38 out of 42 eligible operators (90%). Final plan contains 3 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q33.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q33.native_datafusion/extended.txt index 38b3efb123..6afaa68137 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q33.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q33.native_datafusion/extended.txt @@ -1,112 +1,107 @@ -TakeOrderedAndProject -+- HashAggregate - +- CometNativeColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Union - :- HashAggregate - : +- CometNativeColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- Filter - : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.customer_address - : +- BroadcastExchange - : +- CometNativeColumnarToRow - : +- CometBroadcastHashJoin - : :- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.item - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.item - :- HashAggregate - : +- CometNativeColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- Filter - : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- ReusedSubquery - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.customer_address - : +- BroadcastExchange - : +- CometNativeColumnarToRow - : +- CometBroadcastHashJoin - : :- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.item - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.item - +- HashAggregate - +- CometNativeColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Filter - : : : +- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : +- ReusedSubquery - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- CometNativeColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.customer_address - +- BroadcastExchange - +- CometNativeColumnarToRow - +- CometBroadcastHashJoin - :- CometFilter - : +- CometNativeScan parquet spark_catalog.default.item - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometNativeScan parquet spark_catalog.default.item +CometNativeColumnarToRow ++- CometTakeOrderedAndProject + +- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometUnion + :- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometNativeColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.customer_address + : +- CometBroadcastExchange + : +- CometBroadcastHashJoin + : :- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.item + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.item + :- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometNativeColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.customer_address + : +- CometBroadcastExchange + : +- CometBroadcastHashJoin + : :- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.item + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.item + +- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.web_sales + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometNativeColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.customer_address + +- CometBroadcastExchange + +- CometBroadcastHashJoin + :- CometFilter + : +- CometNativeScan parquet spark_catalog.default.item + +- CometBroadcastExchange + +- CometProject + +- CometFilter + +- CometNativeScan parquet spark_catalog.default.item -Comet accelerated 46 out of 93 eligible operators (49%). Final plan contains 17 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 95 out of 101 eligible operators (94%). Final plan contains 4 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q34.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q34.native_datafusion/extended.txt index 40230582bb..1b61be2937 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q34.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q34.native_datafusion/extended.txt @@ -1,47 +1,41 @@ CometNativeColumnarToRow +- CometSort - +- CometColumnarExchange - +- Project - +- BroadcastHashJoin - :- Filter - : +- HashAggregate - : +- CometNativeColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- Filter - : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.store - : +- BroadcastExchange - : +- CometNativeColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.household_demographics - +- BroadcastExchange - +- CometNativeColumnarToRow - +- CometProject - +- CometFilter - +- CometNativeScan parquet spark_catalog.default.customer + +- CometExchange + +- CometProject + +- CometBroadcastHashJoin + :- CometFilter + : +- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometNativeColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.store + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.household_demographics + +- CometBroadcastExchange + +- CometProject + +- CometFilter + +- CometNativeScan parquet spark_catalog.default.customer -Comet accelerated 18 out of 37 eligible operators (48%). Final plan contains 8 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 35 out of 37 eligible operators (94%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q35.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q35.native_datafusion/extended.txt index 40a8ef4a6e..2785c3d0b1 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q35.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q35.native_datafusion/extended.txt @@ -10,46 +10,52 @@ TakeOrderedAndProject : :- Project : : +- Filter : : +- BroadcastHashJoin - : : :- BroadcastHashJoin - : : : :- BroadcastHashJoin - : : : : :- CometNativeColumnarToRow - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.customer - : : : : +- BroadcastExchange - : : : : +- Project - : : : : +- BroadcastHashJoin - : : : : :- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow + : : :- BroadcastHashJoin [COMET: Unsupported join type ExistenceJoin(exists#1)] + : : : :- CometNativeColumnarToRow + : : : : +- CometBroadcastHashJoin + : : : : :- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.customer + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometNativeScan parquet spark_catalog.default.store_sales + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometNativeColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange : : : : +- CometProject : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- BroadcastExchange - : : : +- Project - : : : +- BroadcastHashJoin - : : : :- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- ReusedSubquery - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow + : : : +- CometNativeColumnarToRow + : : : +- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometNativeScan parquet spark_catalog.default.web_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometNativeColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- BroadcastExchange - : : +- Project - : : +- BroadcastHashJoin - : : :- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : +- ReusedSubquery - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow + : : +- CometNativeColumnarToRow + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometNativeColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange : : +- CometProject : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -64,4 +70,4 @@ TakeOrderedAndProject +- CometFilter +- CometNativeScan parquet spark_catalog.default.customer_demographics -Comet accelerated 21 out of 54 eligible operators (38%). Final plan contains 11 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 41 out of 62 eligible operators (66%). Final plan contains 9 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q36.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q36.native_datafusion/extended.txt index aa40ed68cc..05ce793d8d 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q36.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q36.native_datafusion/extended.txt @@ -3,41 +3,36 @@ TakeOrderedAndProject +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] +- CometNativeColumnarToRow +- CometSort - +- CometColumnarExchange - +- HashAggregate - +- CometNativeColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Expand - +- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Filter - : : : +- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- CometNativeColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.item - +- BroadcastExchange - +- CometNativeColumnarToRow - +- CometProject - +- CometFilter - +- CometNativeScan parquet spark_catalog.default.store + +- CometExchange + +- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometExpand + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometNativeColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.item + +- CometBroadcastExchange + +- CometProject + +- CometFilter + +- CometNativeScan parquet spark_catalog.default.store -Comet accelerated 15 out of 34 eligible operators (44%). Final plan contains 7 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 29 out of 34 eligible operators (85%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q37.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q37.native_datafusion/extended.txt index 296b2c8117..16e3a70d9b 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q37.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q37.native_datafusion/extended.txt @@ -1,38 +1,34 @@ CometNativeColumnarToRow +- CometTakeOrderedAndProject +- CometHashAggregate - +- CometColumnarExchange - +- HashAggregate - +- Project - +- BroadcastHashJoin - :- BroadcastExchange - : +- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.item - : : +- BroadcastExchange - : : +- Project - : : +- Filter - : : +- ColumnarToRow - : : +- Scan parquet spark_catalog.default.inventory [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- CometNativeColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.date_dim - +- CometNativeColumnarToRow - +- CometProject - +- CometFilter - +- CometNativeScan parquet spark_catalog.default.catalog_sales + +- CometExchange + +- CometHashAggregate + +- CometProject + +- CometBroadcastHashJoin + :- CometBroadcastExchange + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.item + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.inventory + : : +- SubqueryBroadcast + : : +- BroadcastExchange + : : +- CometNativeColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.date_dim + +- CometProject + +- CometFilter + +- CometNativeScan parquet spark_catalog.default.catalog_sales -Comet accelerated 15 out of 30 eligible operators (50%). Final plan contains 6 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 28 out of 30 eligible operators (93%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q38.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q38.native_datafusion/extended.txt index 6d9f5c7a17..29422273c5 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q38.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q38.native_datafusion/extended.txt @@ -6,74 +6,75 @@ CometNativeColumnarToRow +- CometBroadcastHashJoin :- CometBroadcastHashJoin : :- CometHashAggregate - : : +- CometColumnarExchange - : : +- HashAggregate - : : +- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- Filter - : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.customer + : : +- CometExchange + : : +- CometHashAggregate + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometNativeColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.customer : +- CometBroadcastExchange : +- CometHashAggregate - : +- CometColumnarExchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Filter - : : : +- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : +- ReusedSubquery - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- CometNativeColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.customer + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometNativeColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.customer +- CometBroadcastExchange +- CometHashAggregate - +- CometColumnarExchange - +- HashAggregate - +- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- Filter - : : +- ColumnarToRow - : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : +- ReusedSubquery - : +- BroadcastExchange - : +- CometNativeColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.date_dim - +- BroadcastExchange - +- CometNativeColumnarToRow - +- CometProject - +- CometFilter - +- CometNativeScan parquet spark_catalog.default.customer + +- CometExchange + +- CometHashAggregate + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.web_sales + : : +- SubqueryBroadcast + : : +- BroadcastExchange + : : +- CometNativeColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.date_dim + +- CometBroadcastExchange + +- CometProject + +- CometFilter + +- CometNativeScan parquet spark_catalog.default.customer -Comet accelerated 35 out of 66 eligible operators (53%). Final plan contains 11 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 68 out of 74 eligible operators (91%). Final plan contains 4 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q39a.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q39a.native_datafusion/extended.txt index d9cec17b28..eba1ff4dcd 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q39a.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q39a.native_datafusion/extended.txt @@ -1,75 +1,65 @@ CometNativeColumnarToRow +- CometSort - +- CometColumnarExchange - +- BroadcastHashJoin - :- Project - : +- Filter - : +- HashAggregate - : +- CometNativeColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- Filter - : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.inventory [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.item - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.warehouse - : +- BroadcastExchange - : +- CometNativeColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.date_dim - +- BroadcastExchange - +- Project - +- Filter - +- HashAggregate - +- CometNativeColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Filter - : : : +- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.inventory [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.item - : +- BroadcastExchange - : +- CometNativeColumnarToRow - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.warehouse - +- BroadcastExchange - +- CometNativeColumnarToRow - +- CometProject - +- CometFilter - +- CometNativeScan parquet spark_catalog.default.date_dim + +- CometExchange + +- CometBroadcastHashJoin + :- CometProject + : +- CometFilter + : +- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.inventory + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometNativeColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.item + : : +- CometBroadcastExchange + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.warehouse + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.date_dim + +- CometBroadcastExchange + +- CometProject + +- CometFilter + +- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.inventory + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometNativeColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.item + : +- CometBroadcastExchange + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.warehouse + +- CometBroadcastExchange + +- CometProject + +- CometFilter + +- CometNativeScan parquet spark_catalog.default.date_dim -Comet accelerated 24 out of 60 eligible operators (40%). Final plan contains 13 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 56 out of 60 eligible operators (93%). Final plan contains 3 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q39b.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q39b.native_datafusion/extended.txt index d9cec17b28..eba1ff4dcd 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q39b.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q39b.native_datafusion/extended.txt @@ -1,75 +1,65 @@ CometNativeColumnarToRow +- CometSort - +- CometColumnarExchange - +- BroadcastHashJoin - :- Project - : +- Filter - : +- HashAggregate - : +- CometNativeColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- Filter - : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.inventory [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.item - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.warehouse - : +- BroadcastExchange - : +- CometNativeColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.date_dim - +- BroadcastExchange - +- Project - +- Filter - +- HashAggregate - +- CometNativeColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Filter - : : : +- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.inventory [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.item - : +- BroadcastExchange - : +- CometNativeColumnarToRow - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.warehouse - +- BroadcastExchange - +- CometNativeColumnarToRow - +- CometProject - +- CometFilter - +- CometNativeScan parquet spark_catalog.default.date_dim + +- CometExchange + +- CometBroadcastHashJoin + :- CometProject + : +- CometFilter + : +- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.inventory + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometNativeColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.item + : : +- CometBroadcastExchange + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.warehouse + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.date_dim + +- CometBroadcastExchange + +- CometProject + +- CometFilter + +- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.inventory + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometNativeColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.item + : +- CometBroadcastExchange + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.warehouse + +- CometBroadcastExchange + +- CometProject + +- CometFilter + +- CometNativeScan parquet spark_catalog.default.date_dim -Comet accelerated 24 out of 60 eligible operators (40%). Final plan contains 13 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 56 out of 60 eligible operators (93%). Final plan contains 3 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q4.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q4.native_datafusion/extended.txt index 737217ff91..b2914584b4 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q4.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q4.native_datafusion/extended.txt @@ -1,154 +1,147 @@ -TakeOrderedAndProject -+- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- BroadcastHashJoin - : : : : :- Filter - : : : : : +- HashAggregate - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometColumnarExchange - : : : : : +- HashAggregate - : : : : : +- Project - : : : : : +- BroadcastHashJoin - : : : : : :- Project - : : : : : : +- BroadcastHashJoin - : : : : : : :- CometNativeColumnarToRow - : : : : : : : +- CometProject - : : : : : : : +- CometFilter - : : : : : : : +- CometNativeScan parquet spark_catalog.default.customer - : : : : : : +- BroadcastExchange - : : : : : : +- Filter - : : : : : : +- ColumnarToRow - : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometNativeColumnarToRow - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : : +- BroadcastExchange - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : +- BroadcastExchange - : : : : +- HashAggregate - : : : : +- CometNativeColumnarToRow - : : : : +- CometColumnarExchange - : : : : +- HashAggregate - : : : : +- Project - : : : : +- BroadcastHashJoin - : : : : :- Project - : : : : : +- BroadcastHashJoin - : : : : : :- CometNativeColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.customer - : : : : : +- BroadcastExchange - : : : : : +- Filter - : : : : : +- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- Filter - : : : +- HashAggregate - : : : +- CometNativeColumnarToRow - : : : +- CometColumnarExchange - : : : +- HashAggregate - : : : +- Project - : : : +- BroadcastHashJoin - : : : :- Project - : : : : +- BroadcastHashJoin - : : : : :- CometNativeColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.customer - : : : : +- BroadcastExchange - : : : : +- Filter - : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- ReusedSubquery - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- HashAggregate - : : +- CometNativeColumnarToRow - : : +- CometColumnarExchange - : : +- HashAggregate - : : +- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.customer - : : : +- BroadcastExchange - : : : +- Filter - : : : +- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : +- ReusedSubquery - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- Filter - : +- HashAggregate - : +- CometNativeColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.customer - : : +- BroadcastExchange - : : +- Filter - : : +- ColumnarToRow - : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : +- ReusedSubquery - : +- BroadcastExchange - : +- CometNativeColumnarToRow - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.date_dim - +- BroadcastExchange - +- HashAggregate - +- CometNativeColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- CometNativeColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.customer - : +- BroadcastExchange - : +- Filter - : +- ColumnarToRow - : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : +- ReusedSubquery - +- BroadcastExchange - +- CometNativeColumnarToRow - +- CometFilter - +- CometNativeScan parquet spark_catalog.default.date_dim +CometNativeColumnarToRow ++- CometTakeOrderedAndProject + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometBroadcastHashJoin + : : : : :- CometFilter + : : : : : +- CometHashAggregate + : : : : : +- CometExchange + : : : : : +- CometHashAggregate + : : : : : +- CometProject + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometProject + : : : : : : +- CometBroadcastHashJoin + : : : : : : :- CometProject + : : : : : : : +- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.customer + : : : : : : +- CometBroadcastExchange + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : : : +- SubqueryBroadcast + : : : : : : +- BroadcastExchange + : : : : : : +- CometNativeColumnarToRow + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- CometBroadcastExchange + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometHashAggregate + : : : : +- CometExchange + : : : : +- CometHashAggregate + : : : : +- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometProject + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.customer + : : : : : +- CometBroadcastExchange + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometNativeColumnarToRow + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometFilter + : : : +- CometHashAggregate + : : : +- CometExchange + : : : +- CometHashAggregate + : : : +- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.customer + : : : : +- CometBroadcastExchange + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometNativeColumnarToRow + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometHashAggregate + : : +- CometExchange + : : +- CometHashAggregate + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.customer + : : : +- CometBroadcastExchange + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometNativeColumnarToRow + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometFilter + : +- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.customer + : : +- CometBroadcastExchange + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.web_sales + : : +- SubqueryBroadcast + : : +- BroadcastExchange + : : +- CometNativeColumnarToRow + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.date_dim + +- CometBroadcastExchange + +- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.customer + : +- CometBroadcastExchange + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.web_sales + : +- SubqueryBroadcast + : +- BroadcastExchange + : +- CometNativeColumnarToRow + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.date_dim + +- CometBroadcastExchange + +- CometFilter + +- CometNativeScan parquet spark_catalog.default.date_dim -Comet accelerated 40 out of 126 eligible operators (31%). Final plan contains 26 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 126 out of 138 eligible operators (91%). Final plan contains 7 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q40.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q40.native_datafusion/extended.txt index f1d98d3059..9b00a2c6a4 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q40.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q40.native_datafusion/extended.txt @@ -12,15 +12,14 @@ CometNativeColumnarToRow : : :- CometProject : : : +- CometSortMergeJoin : : : :- CometSort - : : : : +- CometColumnarExchange - : : : : +- Filter - : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometExchange + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometNativeColumnarToRow + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- CometSort : : : +- CometExchange : : : +- CometProject @@ -38,4 +37,4 @@ CometNativeColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.date_dim -Comet accelerated 32 out of 36 eligible operators (88%). Final plan contains 3 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 34 out of 36 eligible operators (94%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q45.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q45.native_datafusion/extended.txt index 5b686ba177..7292a238d5 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q45.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q45.native_datafusion/extended.txt @@ -5,40 +5,36 @@ TakeOrderedAndProject +- HashAggregate +- Project +- Filter - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- Project - : : : : +- BroadcastHashJoin - : : : : :- Filter - : : : : : +- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.customer - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.customer_address - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- CometNativeColumnarToRow + +- BroadcastHashJoin [COMET: Unsupported join type ExistenceJoin(exists#1)] + :- CometNativeColumnarToRow + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.web_sales + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometNativeColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.customer + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.customer_address + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange : +- CometProject : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.item @@ -48,4 +44,4 @@ TakeOrderedAndProject +- CometFilter +- CometNativeScan parquet spark_catalog.default.item -Comet accelerated 18 out of 41 eligible operators (43%). Final plan contains 8 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 32 out of 41 eligible operators (78%). Final plan contains 4 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q46.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q46.native_datafusion/extended.txt index cebcd85144..dbdaaa275a 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q46.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q46.native_datafusion/extended.txt @@ -1,56 +1,49 @@ -TakeOrderedAndProject -+- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- HashAggregate - : : +- CometNativeColumnarToRow - : : +- CometColumnarExchange - : : +- HashAggregate - : : +- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- Project - : : : : +- BroadcastHashJoin - : : : : :- Project - : : : : : +- BroadcastHashJoin - : : : : : :- Filter - : : : : : : +- ColumnarToRow - : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometNativeColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : : +- BroadcastExchange - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.store - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.household_demographics - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.customer_address - : +- BroadcastExchange - : +- CometNativeColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.customer - +- BroadcastExchange - +- CometNativeColumnarToRow +CometNativeColumnarToRow ++- CometTakeOrderedAndProject + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometHashAggregate + : : +- CometExchange + : : +- CometHashAggregate + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometProject + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : : : +- SubqueryBroadcast + : : : : : : +- BroadcastExchange + : : : : : : +- CometNativeColumnarToRow + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- CometBroadcastExchange + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.store + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.household_demographics + : : +- CometBroadcastExchange + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.customer_address + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.customer + +- CometBroadcastExchange +- CometFilter +- CometNativeScan parquet spark_catalog.default.customer_address -Comet accelerated 20 out of 45 eligible operators (44%). Final plan contains 9 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 43 out of 45 eligible operators (95%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q47.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q47.native_datafusion/extended.txt index ab44bbef5c..0886059452 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q47.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q47.native_datafusion/extended.txt @@ -10,111 +10,96 @@ TakeOrderedAndProject : : +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] : : +- CometNativeColumnarToRow : : +- CometSort - : : +- CometColumnarExchange - : : +- HashAggregate - : : +- CometNativeColumnarToRow - : : +- CometColumnarExchange - : : +- HashAggregate - : : +- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- Project - : : : : +- BroadcastHashJoin - : : : : :- CometNativeColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : +- BroadcastExchange - : : : : +- Filter - : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.store + : : +- CometExchange + : : +- CometHashAggregate + : : +- CometExchange + : : +- CometHashAggregate + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : : +- CometBroadcastExchange + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometNativeColumnarToRow + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.store : +- BroadcastExchange : +- Project : +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] : +- CometNativeColumnarToRow : +- CometSort - : +- CometColumnarExchange - : +- HashAggregate - : +- CometNativeColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : +- BroadcastExchange - : : : +- Filter - : : : +- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- CometNativeColumnarToRow - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.store + : +- CometExchange + : +- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : +- CometBroadcastExchange + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometNativeColumnarToRow + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.store +- BroadcastExchange +- Project +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] +- CometNativeColumnarToRow +- CometSort - +- CometColumnarExchange - +- HashAggregate - +- CometNativeColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.item - : : +- BroadcastExchange - : : +- Filter - : : +- ColumnarToRow - : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- CometNativeColumnarToRow - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.date_dim - +- BroadcastExchange - +- CometNativeColumnarToRow - +- CometFilter - +- CometNativeScan parquet spark_catalog.default.store + +- CometExchange + +- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.item + : : +- CometBroadcastExchange + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : +- SubqueryBroadcast + : : +- BroadcastExchange + : : +- CometNativeColumnarToRow + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.date_dim + +- CometBroadcastExchange + +- CometFilter + +- CometNativeScan parquet spark_catalog.default.store -Comet accelerated 36 out of 97 eligible operators (37%). Final plan contains 21 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 75 out of 97 eligible operators (77%). Final plan contains 6 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q48.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q48.native_datafusion/extended.txt index 5e6a26863a..75a1c725c2 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q48.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q48.native_datafusion/extended.txt @@ -1,42 +1,37 @@ -HashAggregate -+- CometNativeColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- Filter - : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.store - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.customer_demographics - : +- BroadcastExchange - : +- CometNativeColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.customer_address - +- BroadcastExchange - +- CometNativeColumnarToRow - +- CometProject - +- CometFilter - +- CometNativeScan parquet spark_catalog.default.date_dim +CometNativeColumnarToRow ++- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometNativeColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.store + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.customer_demographics + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.customer_address + +- CometBroadcastExchange + +- CometProject + +- CometFilter + +- CometNativeScan parquet spark_catalog.default.date_dim -Comet accelerated 15 out of 33 eligible operators (45%). Final plan contains 7 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 31 out of 33 eligible operators (93%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q49.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q49.native_datafusion/extended.txt index 0090739d55..6d7dadcde9 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q49.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q49.native_datafusion/extended.txt @@ -11,35 +11,31 @@ CometNativeColumnarToRow : +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] : +- CometNativeColumnarToRow : +- CometSort - : +- CometColumnarExchange - : +- HashAggregate - : +- CometNativeColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- BroadcastExchange - : : : +- Project - : : : +- Filter - : : : +- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- CometNativeColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.web_returns - : +- BroadcastExchange - : +- CometNativeColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometExchange + : +- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.web_sales + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometNativeColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.web_returns + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.date_dim :- Project : +- Filter : +- Window @@ -47,30 +43,31 @@ CometNativeColumnarToRow : +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] : +- CometNativeColumnarToRow : +- CometSort - : +- CometColumnarExchange - : +- HashAggregate - : +- CometNativeColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- BroadcastExchange - : : : +- Project - : : : +- Filter - : : : +- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : +- ReusedSubquery - : : +- CometNativeColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.catalog_returns - : +- BroadcastExchange - : +- CometNativeColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometExchange + : +- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometNativeColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.catalog_returns + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.date_dim +- Project +- Filter +- Window @@ -78,29 +75,30 @@ CometNativeColumnarToRow +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] +- CometNativeColumnarToRow +- CometSort - +- CometColumnarExchange - +- HashAggregate - +- CometNativeColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- BroadcastExchange - : : +- Project - : : +- Filter - : : +- ColumnarToRow - : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : +- ReusedSubquery - : +- CometNativeColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.store_returns - +- BroadcastExchange - +- CometNativeColumnarToRow - +- CometProject - +- CometFilter - +- CometNativeScan parquet spark_catalog.default.date_dim + +- CometExchange + +- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : +- SubqueryBroadcast + : : +- BroadcastExchange + : : +- CometNativeColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.store_returns + +- CometBroadcastExchange + +- CometProject + +- CometFilter + +- CometNativeScan parquet spark_catalog.default.date_dim -Comet accelerated 33 out of 87 eligible operators (37%). Final plan contains 17 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 72 out of 95 eligible operators (75%). Final plan contains 7 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q5.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q5.native_datafusion/extended.txt index 15ad014521..719229a9a3 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q5.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q5.native_datafusion/extended.txt @@ -1,106 +1,115 @@ -TakeOrderedAndProject -+- HashAggregate - +- CometNativeColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Expand - +- Union - :- HashAggregate - : +- CometNativeColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Union - : : : :- Project - : : : : +- Filter - : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- Project - : : : +- Filter - : : : +- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.store_returns [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : +- ReusedSubquery - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- CometNativeColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.store - :- HashAggregate - : +- CometNativeColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Union - : : : :- Project - : : : : +- Filter - : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- ReusedSubquery - : : : +- Project - : : : +- Filter - : : : +- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.catalog_returns [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : +- ReusedSubquery - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- CometNativeColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.catalog_page - +- HashAggregate - +- CometNativeColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- Union - : : :- Project - : : : +- Filter - : : : +- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : +- ReusedSubquery - : : +- Project - : : +- BroadcastHashJoin - : : :- BroadcastExchange - : : : +- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.web_returns [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : +- ReusedSubquery - : : +- CometNativeColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.web_sales - : +- BroadcastExchange - : +- CometNativeColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.date_dim - +- BroadcastExchange - +- CometNativeColumnarToRow - +- CometProject - +- CometFilter - +- CometNativeScan parquet spark_catalog.default.web_site +CometNativeColumnarToRow ++- CometTakeOrderedAndProject + +- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometExpand + +- CometUnion + :- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometUnion + : : : :- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometNativeColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.store_returns + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometNativeColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.store + :- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometUnion + : : : :- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometNativeColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.catalog_returns + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometNativeColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.catalog_page + +- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometUnion + : : :- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.web_sales + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometNativeColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometBroadcastExchange + : : : +- CometNativeScan parquet spark_catalog.default.web_returns + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometNativeColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.web_sales + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.date_dim + +- CometBroadcastExchange + +- CometProject + +- CometFilter + +- CometNativeScan parquet spark_catalog.default.web_site -Comet accelerated 28 out of 86 eligible operators (32%). Final plan contains 18 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 94 out of 106 eligible operators (88%). Final plan contains 7 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q50.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q50.native_datafusion/extended.txt index ab41730455..c013381de0 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q50.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q50.native_datafusion/extended.txt @@ -1,42 +1,37 @@ -TakeOrderedAndProject -+- HashAggregate - +- CometNativeColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- CometNativeColumnarToRow - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : +- BroadcastExchange - : : : +- Filter - : : : +- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.store_returns [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.store - : +- BroadcastExchange - : +- CometNativeColumnarToRow - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.date_dim - +- BroadcastExchange - +- CometNativeColumnarToRow - +- CometProject - +- CometFilter - +- CometNativeScan parquet spark_catalog.default.date_dim +CometNativeColumnarToRow ++- CometTakeOrderedAndProject + +- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : +- CometBroadcastExchange + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.store_returns + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometNativeColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.store + : +- CometBroadcastExchange + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.date_dim + +- CometBroadcastExchange + +- CometProject + +- CometFilter + +- CometNativeScan parquet spark_catalog.default.date_dim -Comet accelerated 14 out of 33 eligible operators (42%). Final plan contains 7 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 31 out of 33 eligible operators (93%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q51.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q51.native_datafusion/extended.txt index db4e38a1bb..425e772118 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q51.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q51.native_datafusion/extended.txt @@ -12,48 +12,47 @@ TakeOrderedAndProject : +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] : +- CometNativeColumnarToRow : +- CometSort - : +- CometColumnarExchange - : +- HashAggregate - : +- CometNativeColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- Filter - : : +- ColumnarToRow - : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- CometNativeColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometExchange + : +- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.web_sales + : : +- SubqueryBroadcast + : : +- BroadcastExchange + : : +- CometNativeColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.date_dim +- CometSort +- CometColumnarExchange +- Project +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] +- CometNativeColumnarToRow +- CometSort - +- CometColumnarExchange - +- HashAggregate - +- CometNativeColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Project - +- BroadcastHashJoin - :- Filter - : +- ColumnarToRow - : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : +- ReusedSubquery - +- BroadcastExchange - +- CometNativeColumnarToRow - +- CometProject - +- CometFilter - +- CometNativeScan parquet spark_catalog.default.date_dim + +- CometExchange + +- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometProject + +- CometBroadcastHashJoin + :- CometFilter + : +- CometNativeScan parquet spark_catalog.default.store_sales + : +- SubqueryBroadcast + : +- BroadcastExchange + : +- CometNativeColumnarToRow + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.date_dim + +- CometBroadcastExchange + +- CometProject + +- CometFilter + +- CometNativeScan parquet spark_catalog.default.date_dim -Comet accelerated 23 out of 47 eligible operators (48%). Final plan contains 10 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 40 out of 51 eligible operators (78%). Final plan contains 5 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q53.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q53.native_datafusion/extended.txt index 5c7a26057d..936cde052f 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q53.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q53.native_datafusion/extended.txt @@ -4,39 +4,34 @@ TakeOrderedAndProject +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] +- CometNativeColumnarToRow +- CometSort - +- CometColumnarExchange - +- HashAggregate - +- CometNativeColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.item - : : +- BroadcastExchange - : : +- Filter - : : +- ColumnarToRow - : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- CometNativeColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.date_dim - +- BroadcastExchange - +- CometNativeColumnarToRow - +- CometFilter - +- CometNativeScan parquet spark_catalog.default.store + +- CometExchange + +- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.item + : : +- CometBroadcastExchange + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : +- SubqueryBroadcast + : : +- BroadcastExchange + : : +- CometNativeColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.date_dim + +- CometBroadcastExchange + +- CometFilter + +- CometNativeScan parquet spark_catalog.default.store -Comet accelerated 14 out of 33 eligible operators (42%). Final plan contains 7 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 27 out of 33 eligible operators (81%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q54.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q54.native_datafusion/extended.txt index 8da62eb377..e402b1ddb0 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q54.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q54.native_datafusion/extended.txt @@ -1,120 +1,128 @@ -TakeOrderedAndProject -+- HashAggregate - +- CometNativeColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- HashAggregate - +- CometNativeColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- CometNativeColumnarToRow - : : : : +- CometHashAggregate - : : : : +- CometColumnarExchange - : : : : +- HashAggregate - : : : : +- Project - : : : : +- BroadcastHashJoin - : : : : :- Project - : : : : : +- BroadcastHashJoin - : : : : : :- Project - : : : : : : +- BroadcastHashJoin - : : : : : : :- Union - : : : : : : : :- Project - : : : : : : : : +- Filter - : : : : : : : : +- ColumnarToRow - : : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : : : : +- SubqueryBroadcast - : : : : : : : : +- BroadcastExchange - : : : : : : : : +- CometNativeColumnarToRow - : : : : : : : : +- CometProject - : : : : : : : : +- CometFilter - : : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : : : : +- Project - : : : : : : : +- Filter - : : : : : : : +- ColumnarToRow - : : : : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : : : +- ReusedSubquery - : : : : : : +- BroadcastExchange - : : : : : : +- CometNativeColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : : +- BroadcastExchange - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.customer - : : : +- BroadcastExchange - : : : +- Filter - : : : +- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : : :- ReusedSubquery - : : : : +- ReusedSubquery - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : :- Subquery - : : : : +- CometNativeColumnarToRow - : : : : +- CometHashAggregate - : : : : +- CometExchange - : : : : +- CometHashAggregate - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- Subquery - : : : +- CometNativeColumnarToRow - : : : +- CometHashAggregate - : : : +- CometExchange - : : : +- CometHashAggregate - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.customer_address - : +- BroadcastExchange - : +- CometNativeColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.store - +- BroadcastExchange - +- CometNativeColumnarToRow - +- CometProject - +- CometFilter - : :- ReusedSubquery - : +- ReusedSubquery - +- CometNativeScan parquet spark_catalog.default.date_dim - :- Subquery - : +- CometNativeColumnarToRow - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.date_dim - +- Subquery - +- CometNativeColumnarToRow +CometNativeColumnarToRow ++- CometTakeOrderedAndProject + +- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometHashAggregate + : : : : +- CometExchange + : : : : +- CometHashAggregate + : : : : +- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometProject + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometProject + : : : : : : +- CometBroadcastHashJoin + : : : : : : :- CometUnion + : : : : : : : :- CometProject + : : : : : : : : +- CometFilter + : : : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : : : : : +- SubqueryBroadcast + : : : : : : : : +- BroadcastExchange + : : : : : : : : +- CometNativeColumnarToRow + : : : : : : : : +- CometProject + : : : : : : : : +- CometFilter + : : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : : +- CometProject + : : : : : : : +- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.web_sales + : : : : : : : +- SubqueryBroadcast + : : : : : : : +- BroadcastExchange + : : : : : : : +- CometNativeColumnarToRow + : : : : : : : +- CometProject + : : : : : : : +- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : +- CometBroadcastExchange + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : : : +- CometBroadcastExchange + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.customer + : : : +- CometBroadcastExchange + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometNativeColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : : :- Subquery + : : : : : +- CometNativeColumnarToRow + : : : : : +- CometHashAggregate + : : : : : +- CometExchange + : : : : : +- CometHashAggregate + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- Subquery + : : : : +- CometNativeColumnarToRow + : : : : +- CometHashAggregate + : : : : +- CometExchange + : : : : +- CometHashAggregate + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : :- Subquery + : : : : +- CometNativeColumnarToRow + : : : : +- CometHashAggregate + : : : : +- CometExchange + : : : : +- CometHashAggregate + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- Subquery + : : : +- CometNativeColumnarToRow + : : : +- CometHashAggregate + : : : +- CometExchange + : : : +- CometHashAggregate + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.customer_address + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.store + +- CometBroadcastExchange + +- CometProject + +- CometFilter + : :- ReusedSubquery + : +- ReusedSubquery + +- CometNativeScan parquet spark_catalog.default.date_dim + :- Subquery + : +- CometNativeColumnarToRow + : +- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.date_dim + +- Subquery + +- CometNativeColumnarToRow + +- CometHashAggregate + +- CometExchange +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometFilter - +- CometNativeScan parquet spark_catalog.default.date_dim + +- CometProject + +- CometFilter + +- CometNativeScan parquet spark_catalog.default.date_dim -Comet accelerated 51 out of 100 eligible operators (51%). Final plan contains 18 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 102 out of 116 eligible operators (87%). Final plan contains 10 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q56.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q56.native_datafusion/extended.txt index ad055b48b6..d52c40d57b 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q56.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q56.native_datafusion/extended.txt @@ -1,115 +1,110 @@ -TakeOrderedAndProject -+- HashAggregate - +- CometNativeColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Union - :- HashAggregate - : +- CometNativeColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- Filter - : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.customer_address - : +- BroadcastExchange - : +- CometNativeColumnarToRow - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.item - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.item - :- HashAggregate - : +- CometNativeColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- Filter - : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- ReusedSubquery - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.customer_address - : +- BroadcastExchange - : +- CometNativeColumnarToRow - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.item - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.item - +- HashAggregate - +- CometNativeColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Filter - : : : +- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : +- ReusedSubquery - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- CometNativeColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.customer_address - +- BroadcastExchange - +- CometNativeColumnarToRow - +- CometProject - +- CometBroadcastHashJoin - :- CometFilter - : +- CometNativeScan parquet spark_catalog.default.item - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometNativeScan parquet spark_catalog.default.item +CometNativeColumnarToRow ++- CometTakeOrderedAndProject + +- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometUnion + :- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometNativeColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.customer_address + : +- CometBroadcastExchange + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.item + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.item + :- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometNativeColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.customer_address + : +- CometBroadcastExchange + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.item + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.item + +- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.web_sales + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometNativeColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.customer_address + +- CometBroadcastExchange + +- CometProject + +- CometBroadcastHashJoin + :- CometFilter + : +- CometNativeScan parquet spark_catalog.default.item + +- CometBroadcastExchange + +- CometProject + +- CometFilter + +- CometNativeScan parquet spark_catalog.default.item -Comet accelerated 49 out of 96 eligible operators (51%). Final plan contains 17 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 98 out of 104 eligible operators (94%). Final plan contains 4 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q57.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q57.native_datafusion/extended.txt index 66c510aaa0..024bfeeeea 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q57.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q57.native_datafusion/extended.txt @@ -10,111 +10,96 @@ TakeOrderedAndProject : : +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] : : +- CometNativeColumnarToRow : : +- CometSort - : : +- CometColumnarExchange - : : +- HashAggregate - : : +- CometNativeColumnarToRow - : : +- CometColumnarExchange - : : +- HashAggregate - : : +- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- Project - : : : : +- BroadcastHashJoin - : : : : :- CometNativeColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : +- BroadcastExchange - : : : : +- Filter - : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.call_center + : : +- CometExchange + : : +- CometHashAggregate + : : +- CometExchange + : : +- CometHashAggregate + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : : +- CometBroadcastExchange + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometNativeColumnarToRow + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.call_center : +- BroadcastExchange : +- Project : +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] : +- CometNativeColumnarToRow : +- CometSort - : +- CometColumnarExchange - : +- HashAggregate - : +- CometNativeColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : +- BroadcastExchange - : : : +- Filter - : : : +- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- CometNativeColumnarToRow - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.call_center + : +- CometExchange + : +- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : +- CometBroadcastExchange + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometNativeColumnarToRow + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.call_center +- BroadcastExchange +- Project +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] +- CometNativeColumnarToRow +- CometSort - +- CometColumnarExchange - +- HashAggregate - +- CometNativeColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.item - : : +- BroadcastExchange - : : +- Filter - : : +- ColumnarToRow - : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- CometNativeColumnarToRow - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.date_dim - +- BroadcastExchange - +- CometNativeColumnarToRow - +- CometFilter - +- CometNativeScan parquet spark_catalog.default.call_center + +- CometExchange + +- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.item + : : +- CometBroadcastExchange + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : +- SubqueryBroadcast + : : +- BroadcastExchange + : : +- CometNativeColumnarToRow + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.date_dim + +- CometBroadcastExchange + +- CometFilter + +- CometNativeScan parquet spark_catalog.default.call_center -Comet accelerated 36 out of 97 eligible operators (37%). Final plan contains 21 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 75 out of 97 eligible operators (77%). Final plan contains 6 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q58.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q58.native_datafusion/extended.txt index dd95b1b293..b1e517e7d2 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q58.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q58.native_datafusion/extended.txt @@ -1,127 +1,160 @@ -TakeOrderedAndProject -+- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- Filter - : : +- HashAggregate - : : +- CometNativeColumnarToRow - : : +- CometColumnarExchange - : : +- HashAggregate - : : +- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- Filter - : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : : +- ReusedSubquery - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : +- Subquery - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.item - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : : +- ReusedSubquery - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- Subquery - : : +- CometNativeColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- Filter - : +- HashAggregate - : +- CometNativeColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Filter - : : : +- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : +- ReusedSubquery - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.item - : +- BroadcastExchange - : +- CometNativeColumnarToRow - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : : +- ReusedSubquery - : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- Subquery - : +- CometNativeColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.date_dim - +- BroadcastExchange - +- Filter - +- HashAggregate - +- CometNativeColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- Filter - : : +- ColumnarToRow - : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : +- ReusedSubquery - : +- BroadcastExchange - : +- CometNativeColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.item - +- BroadcastExchange - +- CometNativeColumnarToRow - +- CometProject - +- CometBroadcastHashJoin - :- CometFilter - : +- CometNativeScan parquet spark_catalog.default.date_dim - +- CometBroadcastExchange - +- CometProject - +- CometFilter - : +- ReusedSubquery - +- CometNativeScan parquet spark_catalog.default.date_dim - +- Subquery - +- CometNativeColumnarToRow - +- CometProject - +- CometFilter - +- CometNativeScan parquet spark_catalog.default.date_dim +CometNativeColumnarToRow ++- CometTakeOrderedAndProject + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometFilter + : : +- CometHashAggregate + : : +- CometExchange + : : +- CometHashAggregate + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometNativeColumnarToRow + : : : : +- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : : +- Subquery + : : : : : +- CometNativeColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- Subquery + : : : : +- CometNativeColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.item + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : : +- ReusedSubquery + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- Subquery + : : +- CometNativeColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometFilter + : +- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometNativeColumnarToRow + : : : +- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : : +- Subquery + : : : : +- CometNativeColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- Subquery + : : : +- CometNativeColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.item + : +- CometBroadcastExchange + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : : +- ReusedSubquery + : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- Subquery + : +- CometNativeColumnarToRow + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.date_dim + +- CometBroadcastExchange + +- CometFilter + +- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.web_sales + : : +- SubqueryBroadcast + : : +- BroadcastExchange + : : +- CometNativeColumnarToRow + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : : +- Subquery + : : : +- CometNativeColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- Subquery + : : +- CometNativeColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.item + +- CometBroadcastExchange + +- CometProject + +- CometBroadcastHashJoin + :- CometFilter + : +- CometNativeScan parquet spark_catalog.default.date_dim + +- CometBroadcastExchange + +- CometProject + +- CometFilter + : +- ReusedSubquery + +- CometNativeScan parquet spark_catalog.default.date_dim + +- Subquery + +- CometNativeColumnarToRow + +- CometProject + +- CometFilter + +- CometNativeScan parquet spark_catalog.default.date_dim -Comet accelerated 56 out of 108 eligible operators (51%). Final plan contains 17 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 127 out of 145 eligible operators (87%). Final plan contains 13 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q6.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q6.native_datafusion/extended.txt index 44f11f06fa..d5de400618 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q6.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q6.native_datafusion/extended.txt @@ -1,70 +1,73 @@ -TakeOrderedAndProject -+- Filter - +- HashAggregate - +- CometNativeColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.customer_address - : : : +- CometBroadcastExchange - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.customer - : : +- BroadcastExchange - : : +- Filter - : : +- ColumnarToRow - : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometProject - : : +- CometFilter - : : : +- ReusedSubquery - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- Subquery - : : +- CometNativeColumnarToRow - : : +- CometHashAggregate - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- CometNativeColumnarToRow - : +- CometProject - : +- CometFilter - : : +- ReusedSubquery - : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- Subquery - : +- CometNativeColumnarToRow - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.date_dim - +- BroadcastExchange - +- CometNativeColumnarToRow - +- CometProject - +- CometBroadcastHashJoin - :- CometFilter - : +- CometNativeScan parquet spark_catalog.default.item - +- CometBroadcastExchange - +- CometFilter - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometFilter - +- CometNativeScan parquet spark_catalog.default.item +CometNativeColumnarToRow ++- CometTakeOrderedAndProject + +- CometFilter + +- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.customer_address + : : : +- CometBroadcastExchange + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.customer + : : +- CometBroadcastExchange + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : +- SubqueryBroadcast + : : +- BroadcastExchange + : : +- CometNativeColumnarToRow + : : +- CometProject + : : +- CometFilter + : : : +- Subquery + : : : +- CometNativeColumnarToRow + : : : +- CometHashAggregate + : : : +- CometExchange + : : : +- CometHashAggregate + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- Subquery + : : +- CometNativeColumnarToRow + : : +- CometHashAggregate + : : +- CometExchange + : : +- CometHashAggregate + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : : +- ReusedSubquery + : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- Subquery + : +- CometNativeColumnarToRow + : +- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.date_dim + +- CometBroadcastExchange + +- CometProject + +- CometBroadcastHashJoin + :- CometFilter + : +- CometNativeScan parquet spark_catalog.default.item + +- CometBroadcastExchange + +- CometFilter + +- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometProject + +- CometFilter + +- CometNativeScan parquet spark_catalog.default.item -Comet accelerated 39 out of 60 eligible operators (65%). Final plan contains 8 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 60 out of 66 eligible operators (90%). Final plan contains 5 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q60.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q60.native_datafusion/extended.txt index ad055b48b6..d52c40d57b 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q60.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q60.native_datafusion/extended.txt @@ -1,115 +1,110 @@ -TakeOrderedAndProject -+- HashAggregate - +- CometNativeColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Union - :- HashAggregate - : +- CometNativeColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- Filter - : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.customer_address - : +- BroadcastExchange - : +- CometNativeColumnarToRow - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.item - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.item - :- HashAggregate - : +- CometNativeColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- Filter - : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- ReusedSubquery - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.customer_address - : +- BroadcastExchange - : +- CometNativeColumnarToRow - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.item - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.item - +- HashAggregate - +- CometNativeColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Filter - : : : +- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : +- ReusedSubquery - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- CometNativeColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.customer_address - +- BroadcastExchange - +- CometNativeColumnarToRow - +- CometProject - +- CometBroadcastHashJoin - :- CometFilter - : +- CometNativeScan parquet spark_catalog.default.item - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometNativeScan parquet spark_catalog.default.item +CometNativeColumnarToRow ++- CometTakeOrderedAndProject + +- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometUnion + :- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometNativeColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.customer_address + : +- CometBroadcastExchange + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.item + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.item + :- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometNativeColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.customer_address + : +- CometBroadcastExchange + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.item + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.item + +- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.web_sales + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometNativeColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.customer_address + +- CometBroadcastExchange + +- CometProject + +- CometBroadcastHashJoin + :- CometFilter + : +- CometNativeScan parquet spark_catalog.default.item + +- CometBroadcastExchange + +- CometProject + +- CometFilter + +- CometNativeScan parquet spark_catalog.default.item -Comet accelerated 49 out of 96 eligible operators (51%). Final plan contains 17 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 98 out of 104 eligible operators (94%). Final plan contains 4 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q61.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q61.native_datafusion/extended.txt index 025280d4ca..ef8907c7d8 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q61.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q61.native_datafusion/extended.txt @@ -1,101 +1,93 @@ Project -+- BroadcastNestedLoopJoin - :- HashAggregate - : +- CometNativeColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- Project - : : : : +- BroadcastHashJoin - : : : : :- Project - : : : : : +- BroadcastHashJoin - : : : : : :- Project - : : : : : : +- BroadcastHashJoin - : : : : : : :- Filter - : : : : : : : +- ColumnarToRow - : : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : : : +- SubqueryBroadcast - : : : : : : : +- BroadcastExchange - : : : : : : : +- CometNativeColumnarToRow - : : : : : : : +- CometProject - : : : : : : : +- CometFilter - : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : : : +- BroadcastExchange - : : : : : : +- CometNativeColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.store - : : : : : +- BroadcastExchange - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.promotion - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.customer - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.customer_address - : +- BroadcastExchange - : +- CometNativeColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.item ++- BroadcastNestedLoopJoin [COMET: BroadcastNestedLoopJoin is not supported] + :- CometNativeColumnarToRow + : +- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometProject + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometProject + : : : : : : +- CometBroadcastHashJoin + : : : : : : :- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : : : : +- SubqueryBroadcast + : : : : : : : +- BroadcastExchange + : : : : : : : +- CometNativeColumnarToRow + : : : : : : : +- CometProject + : : : : : : : +- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : +- CometBroadcastExchange + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.store + : : : : : +- CometBroadcastExchange + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.promotion + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.customer + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.customer_address + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.item +- BroadcastExchange - +- HashAggregate - +- CometNativeColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- Project - : : : : +- BroadcastHashJoin - : : : : :- Filter - : : : : : +- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : +- ReusedSubquery - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.store - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.customer - : +- BroadcastExchange - : +- CometNativeColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.customer_address - +- BroadcastExchange - +- CometNativeColumnarToRow - +- CometProject - +- CometFilter - +- CometNativeScan parquet spark_catalog.default.item + +- CometNativeColumnarToRow + +- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometNativeColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.store + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.customer + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.customer_address + +- CometBroadcastExchange + +- CometProject + +- CometFilter + +- CometNativeScan parquet spark_catalog.default.item -Comet accelerated 36 out of 83 eligible operators (43%). Final plan contains 16 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 80 out of 87 eligible operators (91%). Final plan contains 4 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q63.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q63.native_datafusion/extended.txt index 5c7a26057d..936cde052f 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q63.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q63.native_datafusion/extended.txt @@ -4,39 +4,34 @@ TakeOrderedAndProject +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] +- CometNativeColumnarToRow +- CometSort - +- CometColumnarExchange - +- HashAggregate - +- CometNativeColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.item - : : +- BroadcastExchange - : : +- Filter - : : +- ColumnarToRow - : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- CometNativeColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.date_dim - +- BroadcastExchange - +- CometNativeColumnarToRow - +- CometFilter - +- CometNativeScan parquet spark_catalog.default.store + +- CometExchange + +- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.item + : : +- CometBroadcastExchange + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : +- SubqueryBroadcast + : : +- BroadcastExchange + : : +- CometNativeColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.date_dim + +- CometBroadcastExchange + +- CometFilter + +- CometNativeScan parquet spark_catalog.default.store -Comet accelerated 14 out of 33 eligible operators (42%). Final plan contains 7 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 27 out of 33 eligible operators (81%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q64.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q64.native_datafusion/extended.txt index 98db64e5b0..f1fe6fb3e3 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q64.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q64.native_datafusion/extended.txt @@ -40,22 +40,20 @@ CometNativeColumnarToRow : : : : : : : : : : : : : : : :- CometProject : : : : : : : : : : : : : : : : +- CometSortMergeJoin : : : : : : : : : : : : : : : : :- CometSort - : : : : : : : : : : : : : : : : : +- CometColumnarExchange - : : : : : : : : : : : : : : : : : +- Project - : : : : : : : : : : : : : : : : : +- BroadcastHashJoin - : : : : : : : : : : : : : : : : : :- BroadcastExchange - : : : : : : : : : : : : : : : : : : +- Filter - : : : : : : : : : : : : : : : : : : +- ColumnarToRow - : : : : : : : : : : : : : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : : : : : : : : : : : : : : +- SubqueryBroadcast - : : : : : : : : : : : : : : : : : : +- BroadcastExchange - : : : : : : : : : : : : : : : : : : +- CometNativeColumnarToRow - : : : : : : : : : : : : : : : : : : +- CometFilter - : : : : : : : : : : : : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : : : : : : : : : : : : : : +- CometNativeColumnarToRow - : : : : : : : : : : : : : : : : : +- CometProject - : : : : : : : : : : : : : : : : : +- CometFilter - : : : : : : : : : : : : : : : : : +- CometNativeScan parquet spark_catalog.default.store_returns + : : : : : : : : : : : : : : : : : +- CometExchange + : : : : : : : : : : : : : : : : : +- CometProject + : : : : : : : : : : : : : : : : : +- CometBroadcastHashJoin + : : : : : : : : : : : : : : : : : :- CometBroadcastExchange + : : : : : : : : : : : : : : : : : : +- CometFilter + : : : : : : : : : : : : : : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : : : : : : : : : : : : : : : +- SubqueryBroadcast + : : : : : : : : : : : : : : : : : : +- BroadcastExchange + : : : : : : : : : : : : : : : : : : +- CometNativeColumnarToRow + : : : : : : : : : : : : : : : : : : +- CometFilter + : : : : : : : : : : : : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : : : : : : : : : : : : +- CometProject + : : : : : : : : : : : : : : : : : +- CometFilter + : : : : : : : : : : : : : : : : : +- CometNativeScan parquet spark_catalog.default.store_returns : : : : : : : : : : : : : : : : +- CometSort : : : : : : : : : : : : : : : : +- CometProject : : : : : : : : : : : : : : : : +- CometFilter @@ -162,22 +160,20 @@ CometNativeColumnarToRow : : : : : : : : : : : : : : :- CometProject : : : : : : : : : : : : : : : +- CometSortMergeJoin : : : : : : : : : : : : : : : :- CometSort - : : : : : : : : : : : : : : : : +- CometColumnarExchange - : : : : : : : : : : : : : : : : +- Project - : : : : : : : : : : : : : : : : +- BroadcastHashJoin - : : : : : : : : : : : : : : : : :- BroadcastExchange - : : : : : : : : : : : : : : : : : +- Filter - : : : : : : : : : : : : : : : : : +- ColumnarToRow - : : : : : : : : : : : : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : : : : : : : : : : : : : +- SubqueryBroadcast - : : : : : : : : : : : : : : : : : +- BroadcastExchange - : : : : : : : : : : : : : : : : : +- CometNativeColumnarToRow - : : : : : : : : : : : : : : : : : +- CometFilter - : : : : : : : : : : : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : : : : : : : : : : : : : +- CometNativeColumnarToRow - : : : : : : : : : : : : : : : : +- CometProject - : : : : : : : : : : : : : : : : +- CometFilter - : : : : : : : : : : : : : : : : +- CometNativeScan parquet spark_catalog.default.store_returns + : : : : : : : : : : : : : : : : +- CometExchange + : : : : : : : : : : : : : : : : +- CometProject + : : : : : : : : : : : : : : : : +- CometBroadcastHashJoin + : : : : : : : : : : : : : : : : :- CometBroadcastExchange + : : : : : : : : : : : : : : : : : +- CometFilter + : : : : : : : : : : : : : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : : : : : : : : : : : : : : +- SubqueryBroadcast + : : : : : : : : : : : : : : : : : +- BroadcastExchange + : : : : : : : : : : : : : : : : : +- CometNativeColumnarToRow + : : : : : : : : : : : : : : : : : +- CometFilter + : : : : : : : : : : : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : : : : : : : : : : : +- CometProject + : : : : : : : : : : : : : : : : +- CometFilter + : : : : : : : : : : : : : : : : +- CometNativeScan parquet spark_catalog.default.store_returns : : : : : : : : : : : : : : : +- CometSort : : : : : : : : : : : : : : : +- CometProject : : : : : : : : : : : : : : : +- CometFilter @@ -248,4 +244,4 @@ CometNativeColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.item -Comet accelerated 228 out of 242 eligible operators (94%). Final plan contains 7 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 238 out of 242 eligible operators (98%). Final plan contains 3 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q65.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q65.native_datafusion/extended.txt index 73e3aa7046..167a3386c2 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q65.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q65.native_datafusion/extended.txt @@ -1,60 +1,57 @@ -TakeOrderedAndProject -+- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- CometNativeColumnarToRow - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.store - : : +- BroadcastExchange - : : +- Filter - : : +- HashAggregate - : : +- CometNativeColumnarToRow - : : +- CometColumnarExchange - : : +- HashAggregate - : : +- Project - : : +- BroadcastHashJoin - : : :- Filter - : : : +- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- CometNativeColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.item - +- BroadcastExchange - +- Filter - +- HashAggregate - +- CometNativeColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- HashAggregate - +- CometNativeColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Project - +- BroadcastHashJoin - :- Filter - : +- ColumnarToRow - : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : +- ReusedSubquery - +- BroadcastExchange - +- CometNativeColumnarToRow - +- CometProject - +- CometFilter - +- CometNativeScan parquet spark_catalog.default.date_dim +CometNativeColumnarToRow ++- CometTakeOrderedAndProject + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.store + : : +- CometBroadcastExchange + : : +- CometFilter + : : +- CometHashAggregate + : : +- CometExchange + : : +- CometHashAggregate + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometNativeColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.item + +- CometBroadcastExchange + +- CometFilter + +- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometProject + +- CometBroadcastHashJoin + :- CometFilter + : +- CometNativeScan parquet spark_catalog.default.store_sales + : +- SubqueryBroadcast + : +- BroadcastExchange + : +- CometNativeColumnarToRow + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.date_dim + +- CometBroadcastExchange + +- CometProject + +- CometFilter + +- CometNativeScan parquet spark_catalog.default.date_dim -Comet accelerated 17 out of 48 eligible operators (35%). Final plan contains 10 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 48 out of 52 eligible operators (92%). Final plan contains 3 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q66.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q66.native_datafusion/extended.txt index 3f3c6b8d79..a7423e3a59 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q66.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q66.native_datafusion/extended.txt @@ -1,82 +1,74 @@ -TakeOrderedAndProject -+- HashAggregate - +- CometNativeColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Union - :- HashAggregate - : +- CometNativeColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- Project - : : : : +- BroadcastHashJoin - : : : : :- Filter - : : : : : +- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.warehouse - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.time_dim - : +- BroadcastExchange - : +- CometNativeColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.ship_mode - +- HashAggregate - +- CometNativeColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- Filter - : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- ReusedSubquery - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.warehouse - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- CometNativeColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.time_dim - +- BroadcastExchange - +- CometNativeColumnarToRow - +- CometProject - +- CometFilter - +- CometNativeScan parquet spark_catalog.default.ship_mode +CometNativeColumnarToRow ++- CometTakeOrderedAndProject + +- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometUnion + :- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.web_sales + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometNativeColumnarToRow + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.warehouse + : : : +- CometBroadcastExchange + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.time_dim + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.ship_mode + +- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometNativeColumnarToRow + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.warehouse + : : +- CometBroadcastExchange + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.time_dim + +- CometBroadcastExchange + +- CometProject + +- CometFilter + +- CometNativeScan parquet spark_catalog.default.ship_mode -Comet accelerated 27 out of 66 eligible operators (40%). Final plan contains 14 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 65 out of 69 eligible operators (94%). Final plan contains 3 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q67.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q67.native_datafusion/extended.txt index da753c17c1..94c6cc711d 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q67.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q67.native_datafusion/extended.txt @@ -5,42 +5,38 @@ TakeOrderedAndProject +- CometNativeColumnarToRow +- CometSort +- CometColumnarExchange - +- WindowGroupLimit - +- Sort - +- HashAggregate - +- CometNativeColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Expand - +- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Filter - : : : +- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- CometNativeColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.store - +- BroadcastExchange - +- CometNativeColumnarToRow - +- CometProject - +- CometFilter - +- CometNativeScan parquet spark_catalog.default.item + +- WindowGroupLimit [COMET: WindowGroupLimit is not supported] + +- CometNativeColumnarToRow + +- CometSort + +- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometExpand + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometNativeColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.store + +- CometBroadcastExchange + +- CometProject + +- CometFilter + +- CometNativeScan parquet spark_catalog.default.item -Comet accelerated 15 out of 37 eligible operators (40%). Final plan contains 7 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 30 out of 37 eligible operators (81%). Final plan contains 3 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q68.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q68.native_datafusion/extended.txt index cebcd85144..dbdaaa275a 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q68.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q68.native_datafusion/extended.txt @@ -1,56 +1,49 @@ -TakeOrderedAndProject -+- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- HashAggregate - : : +- CometNativeColumnarToRow - : : +- CometColumnarExchange - : : +- HashAggregate - : : +- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- Project - : : : : +- BroadcastHashJoin - : : : : :- Project - : : : : : +- BroadcastHashJoin - : : : : : :- Filter - : : : : : : +- ColumnarToRow - : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometNativeColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : : +- BroadcastExchange - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.store - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.household_demographics - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.customer_address - : +- BroadcastExchange - : +- CometNativeColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.customer - +- BroadcastExchange - +- CometNativeColumnarToRow +CometNativeColumnarToRow ++- CometTakeOrderedAndProject + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometHashAggregate + : : +- CometExchange + : : +- CometHashAggregate + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometProject + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : : : +- SubqueryBroadcast + : : : : : : +- BroadcastExchange + : : : : : : +- CometNativeColumnarToRow + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- CometBroadcastExchange + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.store + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.household_demographics + : : +- CometBroadcastExchange + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.customer_address + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.customer + +- CometBroadcastExchange +- CometFilter +- CometNativeScan parquet spark_catalog.default.customer_address -Comet accelerated 20 out of 45 eligible operators (44%). Final plan contains 9 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 43 out of 45 eligible operators (95%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q69.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q69.native_datafusion/extended.txt index f48c67d6e1..3e0ba48add 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q69.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q69.native_datafusion/extended.txt @@ -9,46 +9,52 @@ TakeOrderedAndProject : +- BroadcastHashJoin : :- Project : : +- BroadcastHashJoin - : : :- BroadcastHashJoin - : : : :- BroadcastHashJoin - : : : : :- CometNativeColumnarToRow - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.customer - : : : : +- BroadcastExchange - : : : : +- Project - : : : : +- BroadcastHashJoin - : : : : :- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow + : : :- BroadcastHashJoin [COMET: BuildRight with LeftAnti is not supported] + : : : :- CometNativeColumnarToRow + : : : : +- CometBroadcastHashJoin + : : : : :- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.customer + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometNativeScan parquet spark_catalog.default.store_sales + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometNativeColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange : : : : +- CometProject : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- BroadcastExchange - : : : +- Project - : : : +- BroadcastHashJoin - : : : :- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- ReusedSubquery - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow + : : : +- CometNativeColumnarToRow + : : : +- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometNativeScan parquet spark_catalog.default.web_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometNativeColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- BroadcastExchange - : : +- Project - : : +- BroadcastHashJoin - : : :- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : +- ReusedSubquery - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow + : : +- CometNativeColumnarToRow + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometNativeColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange : : +- CometProject : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -63,4 +69,4 @@ TakeOrderedAndProject +- CometFilter +- CometNativeScan parquet spark_catalog.default.customer_demographics -Comet accelerated 21 out of 53 eligible operators (39%). Final plan contains 11 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 41 out of 61 eligible operators (67%). Final plan contains 9 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q7.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q7.native_datafusion/extended.txt index b692dfa9be..42206d1be5 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q7.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q7.native_datafusion/extended.txt @@ -1,44 +1,39 @@ -TakeOrderedAndProject -+- HashAggregate - +- CometNativeColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- Filter - : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.customer_demographics - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- CometNativeColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.item - +- BroadcastExchange - +- CometNativeColumnarToRow - +- CometProject - +- CometFilter - +- CometNativeScan parquet spark_catalog.default.promotion +CometNativeColumnarToRow ++- CometTakeOrderedAndProject + +- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometNativeColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.customer_demographics + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.item + +- CometBroadcastExchange + +- CometProject + +- CometFilter + +- CometNativeScan parquet spark_catalog.default.promotion -Comet accelerated 16 out of 35 eligible operators (45%). Final plan contains 7 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 33 out of 35 eligible operators (94%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q70.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q70.native_datafusion/extended.txt index f222353c17..befbf52ae5 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q70.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q70.native_datafusion/extended.txt @@ -11,19 +11,18 @@ TakeOrderedAndProject +- Expand +- Project +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- Filter - : : +- ColumnarToRow - : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- CometNativeColumnarToRow + :- CometNativeColumnarToRow + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : +- SubqueryBroadcast + : : +- BroadcastExchange + : : +- CometNativeColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange : +- CometProject : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -37,29 +36,31 @@ TakeOrderedAndProject +- Project +- Filter +- Window - +- WindowGroupLimit - +- Sort - +- HashAggregate - +- CometNativeColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- Filter - : : +- ColumnarToRow - : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : +- ReusedSubquery - : +- BroadcastExchange - : +- CometNativeColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.store - +- BroadcastExchange - +- CometNativeColumnarToRow - +- CometProject - +- CometFilter - +- CometNativeScan parquet spark_catalog.default.date_dim + +- WindowGroupLimit [COMET: WindowGroupLimit is not supported] + +- CometNativeColumnarToRow + +- CometSort + +- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : +- SubqueryBroadcast + : : +- BroadcastExchange + : : +- CometNativeColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.store + +- CometBroadcastExchange + +- CometProject + +- CometFilter + +- CometNativeScan parquet spark_catalog.default.date_dim -Comet accelerated 18 out of 53 eligible operators (33%). Final plan contains 10 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 37 out of 57 eligible operators (64%). Final plan contains 7 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q71.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q71.native_datafusion/extended.txt index 23844f41f3..7f76dd192d 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q71.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q71.native_datafusion/extended.txt @@ -1,62 +1,63 @@ CometNativeColumnarToRow +- CometSort - +- CometColumnarExchange - +- HashAggregate - +- CometNativeColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.item - : +- Union - : :- Project - : : +- BroadcastHashJoin - : : :- Filter - : : : +- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : :- Project - : : +- BroadcastHashJoin - : : :- Filter - : : : +- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : +- ReusedSubquery - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- Project - : +- BroadcastHashJoin - : :- Filter - : : +- ColumnarToRow - : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : +- ReusedSubquery - : +- BroadcastExchange - : +- CometNativeColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.date_dim - +- BroadcastExchange - +- CometNativeColumnarToRow - +- CometProject - +- CometFilter - +- CometNativeScan parquet spark_catalog.default.time_dim + +- CometExchange + +- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.item + : +- CometUnion + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.web_sales + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometNativeColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometNativeColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : +- SubqueryBroadcast + : : +- BroadcastExchange + : : +- CometNativeColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.date_dim + +- CometBroadcastExchange + +- CometProject + +- CometFilter + +- CometNativeScan parquet spark_catalog.default.time_dim -Comet accelerated 21 out of 49 eligible operators (42%). Final plan contains 11 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 51 out of 57 eligible operators (89%). Final plan contains 4 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q72.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q72.native_datafusion/extended.txt index 62c2b17b7e..623a68ddab 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q72.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q72.native_datafusion/extended.txt @@ -6,77 +6,67 @@ CometNativeColumnarToRow +- CometProject +- CometSortMergeJoin :- CometSort - : +- CometColumnarExchange - : +- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- Project - : : : : +- BroadcastHashJoin - : : : : :- Project - : : : : : +- BroadcastHashJoin - : : : : : :- Project - : : : : : : +- BroadcastHashJoin - : : : : : : :- Project - : : : : : : : +- BroadcastHashJoin - : : : : : : : :- Project - : : : : : : : : +- BroadcastHashJoin - : : : : : : : : :- Project - : : : : : : : : : +- BroadcastHashJoin - : : : : : : : : : :- Filter - : : : : : : : : : : +- ColumnarToRow - : : : : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : : : : : : +- SubqueryBroadcast - : : : : : : : : : : +- BroadcastExchange - : : : : : : : : : : +- CometNativeColumnarToRow - : : : : : : : : : : +- CometProject - : : : : : : : : : : +- CometFilter - : : : : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : : : : : : +- BroadcastExchange - : : : : : : : : : +- CometNativeColumnarToRow - : : : : : : : : : +- CometFilter - : : : : : : : : : +- CometNativeScan parquet spark_catalog.default.inventory - : : : : : : : : +- BroadcastExchange - : : : : : : : : +- CometNativeColumnarToRow - : : : : : : : : +- CometFilter - : : : : : : : : +- CometNativeScan parquet spark_catalog.default.warehouse - : : : : : : : +- BroadcastExchange - : : : : : : : +- CometNativeColumnarToRow - : : : : : : : +- CometFilter - : : : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : : : +- BroadcastExchange - : : : : : : +- CometNativeColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.customer_demographics - : : : : : +- BroadcastExchange - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.household_demographics - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- CometNativeColumnarToRow - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.promotion + : +- CometExchange + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometProject + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometProject + : : : : : : +- CometBroadcastHashJoin + : : : : : : :- CometProject + : : : : : : : +- CometBroadcastHashJoin + : : : : : : : :- CometProject + : : : : : : : : +- CometBroadcastHashJoin + : : : : : : : : :- CometProject + : : : : : : : : : +- CometBroadcastHashJoin + : : : : : : : : : :- CometFilter + : : : : : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : : : : : : : +- SubqueryBroadcast + : : : : : : : : : : +- BroadcastExchange + : : : : : : : : : : +- CometNativeColumnarToRow + : : : : : : : : : : +- CometProject + : : : : : : : : : : +- CometFilter + : : : : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : : : : +- CometBroadcastExchange + : : : : : : : : : +- CometFilter + : : : : : : : : : +- CometNativeScan parquet spark_catalog.default.inventory + : : : : : : : : +- CometBroadcastExchange + : : : : : : : : +- CometFilter + : : : : : : : : +- CometNativeScan parquet spark_catalog.default.warehouse + : : : : : : : +- CometBroadcastExchange + : : : : : : : +- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : : : : +- CometBroadcastExchange + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.customer_demographics + : : : : : +- CometBroadcastExchange + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.household_demographics + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.promotion +- CometSort +- CometExchange +- CometProject +- CometFilter +- CometNativeScan parquet spark_catalog.default.catalog_returns -Comet accelerated 37 out of 68 eligible operators (54%). Final plan contains 12 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 66 out of 68 eligible operators (97%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q73.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q73.native_datafusion/extended.txt index 40230582bb..1b61be2937 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q73.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q73.native_datafusion/extended.txt @@ -1,47 +1,41 @@ CometNativeColumnarToRow +- CometSort - +- CometColumnarExchange - +- Project - +- BroadcastHashJoin - :- Filter - : +- HashAggregate - : +- CometNativeColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- Filter - : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.store - : +- BroadcastExchange - : +- CometNativeColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.household_demographics - +- BroadcastExchange - +- CometNativeColumnarToRow - +- CometProject - +- CometFilter - +- CometNativeScan parquet spark_catalog.default.customer + +- CometExchange + +- CometProject + +- CometBroadcastHashJoin + :- CometFilter + : +- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometNativeColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.store + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.household_demographics + +- CometBroadcastExchange + +- CometProject + +- CometFilter + +- CometNativeScan parquet spark_catalog.default.customer -Comet accelerated 18 out of 37 eligible operators (48%). Final plan contains 8 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 35 out of 37 eligible operators (94%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q74.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q74.native_datafusion/extended.txt index 5d218998be..51a3f5ca94 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q74.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q74.native_datafusion/extended.txt @@ -1,105 +1,98 @@ -TakeOrderedAndProject -+- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- BroadcastHashJoin - : : :- Filter - : : : +- HashAggregate - : : : +- CometNativeColumnarToRow - : : : +- CometColumnarExchange - : : : +- HashAggregate - : : : +- Project - : : : +- BroadcastHashJoin - : : : :- Project - : : : : +- BroadcastHashJoin - : : : : :- CometNativeColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.customer - : : : : +- BroadcastExchange - : : : : +- Filter - : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- HashAggregate - : : +- CometNativeColumnarToRow - : : +- CometColumnarExchange - : : +- HashAggregate - : : +- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.customer - : : : +- BroadcastExchange - : : : +- Filter - : : : +- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- Filter - : +- HashAggregate - : +- CometNativeColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.customer - : : +- BroadcastExchange - : : +- Filter - : : +- ColumnarToRow - : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : +- ReusedSubquery - : +- BroadcastExchange - : +- CometNativeColumnarToRow - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.date_dim - +- BroadcastExchange - +- HashAggregate - +- CometNativeColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- CometNativeColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.customer - : +- BroadcastExchange - : +- Filter - : +- ColumnarToRow - : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : +- ReusedSubquery - +- BroadcastExchange - +- CometNativeColumnarToRow - +- CometFilter - +- CometNativeScan parquet spark_catalog.default.date_dim +CometNativeColumnarToRow ++- CometTakeOrderedAndProject + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometHashAggregate + : : : +- CometExchange + : : : +- CometHashAggregate + : : : +- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.customer + : : : : +- CometBroadcastExchange + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometNativeColumnarToRow + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometHashAggregate + : : +- CometExchange + : : +- CometHashAggregate + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.customer + : : : +- CometBroadcastExchange + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometNativeColumnarToRow + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometFilter + : +- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.customer + : : +- CometBroadcastExchange + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.web_sales + : : +- SubqueryBroadcast + : : +- BroadcastExchange + : : +- CometNativeColumnarToRow + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.date_dim + +- CometBroadcastExchange + +- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.customer + : +- CometBroadcastExchange + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.web_sales + : +- SubqueryBroadcast + : +- BroadcastExchange + : +- CometNativeColumnarToRow + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.date_dim + +- CometBroadcastExchange + +- CometFilter + +- CometNativeScan parquet spark_catalog.default.date_dim -Comet accelerated 28 out of 85 eligible operators (32%). Final plan contains 18 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 83 out of 91 eligible operators (91%). Final plan contains 5 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q75.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q75.native_datafusion/extended.txt index 42106a0ffd..a943308f1e 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q75.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q75.native_datafusion/extended.txt @@ -15,28 +15,25 @@ CometNativeColumnarToRow : :- CometProject : : +- CometSortMergeJoin : : :- CometSort - : : : +- CometColumnarExchange - : : : +- Project - : : : +- BroadcastHashJoin - : : : :- Project - : : : : +- BroadcastHashJoin - : : : : :- Filter - : : : : : +- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometExchange + : : : +- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometNativeColumnarToRow + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : +- CometBroadcastExchange + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- CometSort : : +- CometExchange : : +- CometProject @@ -45,24 +42,25 @@ CometNativeColumnarToRow : :- CometProject : : +- CometSortMergeJoin : : :- CometSort - : : : +- CometColumnarExchange - : : : +- Project - : : : +- BroadcastHashJoin - : : : :- Project - : : : : +- BroadcastHashJoin - : : : : :- Filter - : : : : : +- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : +- ReusedSubquery - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometExchange + : : : +- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometNativeColumnarToRow + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : +- CometBroadcastExchange + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- CometSort : : +- CometExchange : : +- CometProject @@ -71,24 +69,25 @@ CometNativeColumnarToRow : +- CometProject : +- CometSortMergeJoin : :- CometSort - : : +- CometColumnarExchange - : : +- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- Filter - : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- ReusedSubquery - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.item - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometExchange + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.web_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometNativeColumnarToRow + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.item + : : +- CometBroadcastExchange + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim : +- CometSort : +- CometExchange : +- CometProject @@ -107,28 +106,25 @@ CometNativeColumnarToRow :- CometProject : +- CometSortMergeJoin : :- CometSort - : : +- CometColumnarExchange - : : +- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- Filter - : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.item - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometExchange + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometNativeColumnarToRow + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.item + : : +- CometBroadcastExchange + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim : +- CometSort : +- CometExchange : +- CometProject @@ -137,24 +133,25 @@ CometNativeColumnarToRow :- CometProject : +- CometSortMergeJoin : :- CometSort - : : +- CometColumnarExchange - : : +- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- Filter - : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- ReusedSubquery - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.item - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometExchange + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometNativeColumnarToRow + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.item + : : +- CometBroadcastExchange + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim : +- CometSort : +- CometExchange : +- CometProject @@ -163,28 +160,29 @@ CometNativeColumnarToRow +- CometProject +- CometSortMergeJoin :- CometSort - : +- CometColumnarExchange - : +- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Filter - : : : +- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : +- ReusedSubquery - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.item - : +- BroadcastExchange - : +- CometNativeColumnarToRow - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometExchange + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.web_sales + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometNativeColumnarToRow + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.item + : +- CometBroadcastExchange + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.date_dim +- CometSort +- CometExchange +- CometProject +- CometFilter +- CometNativeScan parquet spark_catalog.default.web_returns -Comet accelerated 111 out of 167 eligible operators (66%). Final plan contains 21 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 167 out of 179 eligible operators (93%). Final plan contains 7 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q77.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q77.native_datafusion/extended.txt index 5d2ca1a9d0..835f7a449a 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q77.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q77.native_datafusion/extended.txt @@ -5,131 +5,138 @@ TakeOrderedAndProject +- HashAggregate +- Expand +- Union + :- CometNativeColumnarToRow + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometHashAggregate + : : +- CometExchange + : : +- CometHashAggregate + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometNativeColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.store + : +- CometBroadcastExchange + : +- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.store_returns + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometNativeColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.store :- Project - : +- BroadcastHashJoin - : :- HashAggregate - : : +- CometNativeColumnarToRow - : : +- CometColumnarExchange - : : +- HashAggregate - : : +- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- Filter - : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.store - : +- BroadcastExchange - : +- HashAggregate - : +- CometNativeColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Filter - : : : +- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.store_returns [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : +- ReusedSubquery - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- CometNativeColumnarToRow - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.store - :- Project - : +- BroadcastNestedLoopJoin + : +- BroadcastNestedLoopJoin [COMET: BroadcastNestedLoopJoin is not supported] : :- BroadcastExchange - : : +- HashAggregate - : : +- CometNativeColumnarToRow - : : +- CometColumnarExchange - : : +- HashAggregate - : : +- Project - : : +- BroadcastHashJoin - : : :- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : +- ReusedSubquery - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- HashAggregate - : +- CometNativeColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- ColumnarToRow - : : +- Scan parquet spark_catalog.default.catalog_returns [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : +- ReusedSubquery - : +- BroadcastExchange - : +- CometNativeColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.date_dim - +- Project - +- BroadcastHashJoin - :- HashAggregate - : +- CometNativeColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Filter - : : : +- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : +- ReusedSubquery - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- CometNativeColumnarToRow - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.web_page - +- BroadcastExchange - +- HashAggregate - +- CometNativeColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- Filter - : : +- ColumnarToRow - : : +- Scan parquet spark_catalog.default.web_returns [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : +- ReusedSubquery - : +- BroadcastExchange - : +- CometNativeColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.date_dim - +- BroadcastExchange - +- CometNativeColumnarToRow - +- CometFilter - +- CometNativeScan parquet spark_catalog.default.web_page + : : +- CometNativeColumnarToRow + : : +- CometHashAggregate + : : +- CometExchange + : : +- CometHashAggregate + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometNativeColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometNativeColumnarToRow + : +- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometNativeScan parquet spark_catalog.default.catalog_returns + : : +- SubqueryBroadcast + : : +- BroadcastExchange + : : +- CometNativeColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.date_dim + +- CometNativeColumnarToRow + +- CometProject + +- CometBroadcastHashJoin + :- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.web_sales + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometNativeColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.web_page + +- CometBroadcastExchange + +- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.web_returns + : : +- SubqueryBroadcast + : : +- BroadcastExchange + : : +- CometNativeColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.date_dim + +- CometBroadcastExchange + +- CometFilter + +- CometNativeScan parquet spark_catalog.default.web_page -Comet accelerated 36 out of 109 eligible operators (33%). Final plan contains 24 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 109 out of 129 eligible operators (84%). Final plan contains 11 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q78.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q78.native_datafusion/extended.txt index af4b7000b9..ebfdcc3c62 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q78.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q78.native_datafusion/extended.txt @@ -14,15 +14,14 @@ TakeOrderedAndProject : : : +- CometFilter : : : +- CometSortMergeJoin : : : :- CometSort - : : : : +- CometColumnarExchange - : : : : +- Filter - : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometExchange + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometNativeColumnarToRow + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- CometSort : : : +- CometExchange : : : +- CometProject @@ -42,11 +41,14 @@ TakeOrderedAndProject : : +- CometFilter : : +- CometSortMergeJoin : : :- CometSort - : : : +- CometColumnarExchange - : : : +- Filter - : : : +- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : +- ReusedSubquery + : : : +- CometExchange + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.web_sales + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometNativeColumnarToRow + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- CometSort : : +- CometExchange : : +- CometProject @@ -66,11 +68,14 @@ TakeOrderedAndProject : +- CometFilter : +- CometSortMergeJoin : :- CometSort - : : +- CometColumnarExchange - : : +- Filter - : : +- ColumnarToRow - : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : +- ReusedSubquery + : : +- CometExchange + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : +- SubqueryBroadcast + : : +- BroadcastExchange + : : +- CometNativeColumnarToRow + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim : +- CometSort : +- CometExchange : +- CometProject @@ -80,4 +85,4 @@ TakeOrderedAndProject +- CometFilter +- CometNativeScan parquet spark_catalog.default.date_dim -Comet accelerated 64 out of 76 eligible operators (84%). Final plan contains 5 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 74 out of 82 eligible operators (90%). Final plan contains 4 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q79.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q79.native_datafusion/extended.txt index f24d730d8b..526b86f475 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q79.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q79.native_datafusion/extended.txt @@ -1,44 +1,39 @@ -TakeOrderedAndProject -+- Project - +- BroadcastHashJoin - :- HashAggregate - : +- CometNativeColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- Filter - : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.store - : +- BroadcastExchange - : +- CometNativeColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.household_demographics - +- BroadcastExchange - +- CometNativeColumnarToRow +CometNativeColumnarToRow ++- CometTakeOrderedAndProject + +- CometProject + +- CometBroadcastHashJoin + :- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometNativeColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.store + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.household_demographics + +- CometBroadcastExchange +- CometProject +- CometFilter +- CometNativeScan parquet spark_catalog.default.customer -Comet accelerated 16 out of 35 eligible operators (45%). Final plan contains 7 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 33 out of 35 eligible operators (94%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q8.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q8.native_datafusion/extended.txt index 8c9eb7b8c9..cb04fc12c8 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q8.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q8.native_datafusion/extended.txt @@ -1,56 +1,52 @@ -TakeOrderedAndProject -+- HashAggregate - +- CometNativeColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Filter - : : : +- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- CometNativeColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.store - +- BroadcastExchange - +- CometNativeColumnarToRow - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometBroadcastHashJoin - :- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.customer_address - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.customer_address - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometNativeScan parquet spark_catalog.default.customer +CometNativeColumnarToRow ++- CometTakeOrderedAndProject + +- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometNativeColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.store + +- CometBroadcastExchange + +- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometBroadcastHashJoin + :- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.customer_address + +- CometBroadcastExchange + +- CometProject + +- CometFilter + +- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.customer_address + +- CometBroadcastExchange + +- CometProject + +- CometFilter + +- CometNativeScan parquet spark_catalog.default.customer -Comet accelerated 32 out of 48 eligible operators (66%). Final plan contains 6 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 46 out of 48 eligible operators (95%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q80.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q80.native_datafusion/extended.txt index 6853afccad..003be45ee5 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q80.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q80.native_datafusion/extended.txt @@ -19,16 +19,15 @@ CometNativeColumnarToRow : : : : :- CometProject : : : : : +- CometSortMergeJoin : : : : : :- CometSort - : : : : : : +- CometColumnarExchange - : : : : : : +- Filter - : : : : : : +- ColumnarToRow - : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometNativeColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : +- CometExchange + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : : : +- SubqueryBroadcast + : : : : : : +- BroadcastExchange + : : : : : : +- CometNativeColumnarToRow + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : : +- CometSort : : : : : +- CometExchange : : : : : +- CometProject @@ -64,11 +63,15 @@ CometNativeColumnarToRow : : : : :- CometProject : : : : : +- CometSortMergeJoin : : : : : :- CometSort - : : : : : : +- CometColumnarExchange - : : : : : : +- Filter - : : : : : : +- ColumnarToRow - : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : : +- ReusedSubquery + : : : : : : +- CometExchange + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : : : +- SubqueryBroadcast + : : : : : : +- BroadcastExchange + : : : : : : +- CometNativeColumnarToRow + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : : +- CometSort : : : : : +- CometExchange : : : : : +- CometProject @@ -104,11 +107,15 @@ CometNativeColumnarToRow : : : :- CometProject : : : : +- CometSortMergeJoin : : : : :- CometSort - : : : : : +- CometColumnarExchange - : : : : : +- Filter - : : : : : +- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : +- ReusedSubquery + : : : : : +- CometExchange + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.web_sales + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometNativeColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : +- CometSort : : : : +- CometExchange : : : : +- CometProject @@ -131,4 +138,4 @@ CometNativeColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.promotion -Comet accelerated 117 out of 127 eligible operators (92%). Final plan contains 5 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 129 out of 135 eligible operators (95%). Final plan contains 4 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q81.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q81.native_datafusion/extended.txt index 841b2ca52d..949d50a867 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q81.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q81.native_datafusion/extended.txt @@ -1,75 +1,70 @@ -TakeOrderedAndProject -+- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Filter - : : : +- HashAggregate - : : : +- CometNativeColumnarToRow - : : : +- CometColumnarExchange - : : : +- HashAggregate - : : : +- Project - : : : +- BroadcastHashJoin - : : : :- Project - : : : : +- BroadcastHashJoin - : : : : :- Filter - : : : : : +- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.catalog_returns [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.customer_address - : : +- BroadcastExchange - : : +- Filter - : : +- HashAggregate - : : +- CometNativeColumnarToRow - : : +- CometColumnarExchange - : : +- HashAggregate - : : +- HashAggregate - : : +- CometNativeColumnarToRow - : : +- CometColumnarExchange - : : +- HashAggregate - : : +- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- Filter - : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.catalog_returns [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- ReusedSubquery - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.customer_address - : +- BroadcastExchange - : +- CometNativeColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.customer - +- BroadcastExchange - +- CometNativeColumnarToRow +CometNativeColumnarToRow ++- CometTakeOrderedAndProject + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometHashAggregate + : : : +- CometExchange + : : : +- CometHashAggregate + : : : +- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_returns + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometNativeColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.customer_address + : : +- CometBroadcastExchange + : : +- CometFilter + : : +- CometHashAggregate + : : +- CometExchange + : : +- CometHashAggregate + : : +- CometHashAggregate + : : +- CometExchange + : : +- CometHashAggregate + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.catalog_returns + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometNativeColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.customer_address + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.customer + +- CometBroadcastExchange +- CometProject +- CometFilter +- CometNativeScan parquet spark_catalog.default.customer_address -Comet accelerated 24 out of 61 eligible operators (39%). Final plan contains 12 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 61 out of 65 eligible operators (93%). Final plan contains 3 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q82.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q82.native_datafusion/extended.txt index 8abf5c1fb1..30fca82ec8 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q82.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q82.native_datafusion/extended.txt @@ -1,38 +1,34 @@ CometNativeColumnarToRow +- CometTakeOrderedAndProject +- CometHashAggregate - +- CometColumnarExchange - +- HashAggregate - +- Project - +- BroadcastHashJoin - :- BroadcastExchange - : +- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.item - : : +- BroadcastExchange - : : +- Project - : : +- Filter - : : +- ColumnarToRow - : : +- Scan parquet spark_catalog.default.inventory [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- CometNativeColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.date_dim - +- CometNativeColumnarToRow - +- CometProject - +- CometFilter - +- CometNativeScan parquet spark_catalog.default.store_sales + +- CometExchange + +- CometHashAggregate + +- CometProject + +- CometBroadcastHashJoin + :- CometBroadcastExchange + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.item + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.inventory + : : +- SubqueryBroadcast + : : +- BroadcastExchange + : : +- CometNativeColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.date_dim + +- CometProject + +- CometFilter + +- CometNativeScan parquet spark_catalog.default.store_sales -Comet accelerated 15 out of 30 eligible operators (50%). Final plan contains 6 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 28 out of 30 eligible operators (93%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q83.ansi.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q83.ansi.native_datafusion/extended.txt index 7616ffd2b1..3a16430014 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q83.ansi.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q83.ansi.native_datafusion/extended.txt @@ -1,116 +1,133 @@ -TakeOrderedAndProject -+- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- HashAggregate - : : +- CometNativeColumnarToRow - : : +- CometColumnarExchange - : : +- HashAggregate - : : +- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- Filter - : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.store_returns [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometNativeScan parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.item - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometNativeScan parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- HashAggregate - : +- CometNativeColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Filter - : : : +- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.catalog_returns [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : +- ReusedSubquery - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.item - : +- BroadcastExchange - : +- CometNativeColumnarToRow - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometNativeScan parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.date_dim - +- BroadcastExchange - +- HashAggregate - +- CometNativeColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- Filter - : : +- ColumnarToRow - : : +- Scan parquet spark_catalog.default.web_returns [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : +- ReusedSubquery - : +- BroadcastExchange - : +- CometNativeColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.item - +- BroadcastExchange - +- CometNativeColumnarToRow - +- CometProject - +- CometBroadcastHashJoin - :- CometFilter - : +- CometNativeScan parquet spark_catalog.default.date_dim - +- CometBroadcastExchange - +- CometProject - +- CometBroadcastHashJoin - :- CometNativeScan parquet spark_catalog.default.date_dim - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometNativeScan parquet spark_catalog.default.date_dim +CometNativeColumnarToRow ++- CometTakeOrderedAndProject + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometHashAggregate + : : +- CometExchange + : : +- CometHashAggregate + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.store_returns + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometNativeColumnarToRow + : : : : +- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.item + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.catalog_returns + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometNativeColumnarToRow + : : : +- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.item + : +- CometBroadcastExchange + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.date_dim + +- CometBroadcastExchange + +- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.web_returns + : : +- SubqueryBroadcast + : : +- BroadcastExchange + : : +- CometNativeColumnarToRow + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.item + +- CometBroadcastExchange + +- CometProject + +- CometBroadcastHashJoin + :- CometFilter + : +- CometNativeScan parquet spark_catalog.default.date_dim + +- CometBroadcastExchange + +- CometProject + +- CometBroadcastHashJoin + :- CometNativeScan parquet spark_catalog.default.date_dim + +- CometBroadcastExchange + +- CometProject + +- CometFilter + +- CometNativeScan parquet spark_catalog.default.date_dim -Comet accelerated 60 out of 101 eligible operators (59%). Final plan contains 13 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 121 out of 127 eligible operators (95%). Final plan contains 4 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q85.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q85.native_datafusion/extended.txt index aed7b365be..06537a5eec 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q85.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q85.native_datafusion/extended.txt @@ -1,64 +1,56 @@ -TakeOrderedAndProject -+- HashAggregate - +- CometNativeColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- Project - : : : : +- BroadcastHashJoin - : : : : :- Project - : : : : : +- BroadcastHashJoin - : : : : : :- Project - : : : : : : +- BroadcastHashJoin - : : : : : : :- BroadcastExchange - : : : : : : : +- Filter - : : : : : : : +- ColumnarToRow - : : : : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : : : +- SubqueryBroadcast - : : : : : : : +- BroadcastExchange - : : : : : : : +- CometNativeColumnarToRow - : : : : : : : +- CometProject - : : : : : : : +- CometFilter - : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : : : +- CometNativeColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.web_returns - : : : : : +- BroadcastExchange - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.web_page - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.customer_demographics - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.customer_demographics - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.customer_address - : +- BroadcastExchange - : +- CometNativeColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.date_dim - +- BroadcastExchange - +- CometNativeColumnarToRow - +- CometProject - +- CometFilter - +- CometNativeScan parquet spark_catalog.default.reason +CometNativeColumnarToRow ++- CometTakeOrderedAndProject + +- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometProject + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometProject + : : : : : : +- CometBroadcastHashJoin + : : : : : : :- CometBroadcastExchange + : : : : : : : +- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.web_sales + : : : : : : : +- SubqueryBroadcast + : : : : : : : +- BroadcastExchange + : : : : : : : +- CometNativeColumnarToRow + : : : : : : : +- CometProject + : : : : : : : +- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.web_returns + : : : : : +- CometBroadcastExchange + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.web_page + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.customer_demographics + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.customer_demographics + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.customer_address + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.date_dim + +- CometBroadcastExchange + +- CometProject + +- CometFilter + +- CometNativeScan parquet spark_catalog.default.reason -Comet accelerated 24 out of 52 eligible operators (46%). Final plan contains 10 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 50 out of 52 eligible operators (96%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q86.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q86.native_datafusion/extended.txt index a60a3158a4..c9117e2ae0 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q86.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q86.native_datafusion/extended.txt @@ -3,34 +3,30 @@ TakeOrderedAndProject +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] +- CometNativeColumnarToRow +- CometSort - +- CometColumnarExchange - +- HashAggregate - +- CometNativeColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Expand - +- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- Filter - : : +- ColumnarToRow - : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- CometNativeColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.date_dim - +- BroadcastExchange - +- CometNativeColumnarToRow - +- CometProject - +- CometFilter - +- CometNativeScan parquet spark_catalog.default.item + +- CometExchange + +- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometExpand + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.web_sales + : : +- SubqueryBroadcast + : : +- BroadcastExchange + : : +- CometNativeColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.date_dim + +- CometBroadcastExchange + +- CometProject + +- CometFilter + +- CometNativeScan parquet spark_catalog.default.item -Comet accelerated 12 out of 28 eligible operators (42%). Final plan contains 6 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 23 out of 28 eligible operators (82%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q87.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q87.native_datafusion/extended.txt index 7a9e62d57c..6676ec7a4b 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q87.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q87.native_datafusion/extended.txt @@ -7,76 +7,77 @@ HashAggregate :- BroadcastHashJoin [COMET: BuildRight with LeftAnti is not supported] : :- CometNativeColumnarToRow : : +- CometHashAggregate - : : +- CometColumnarExchange - : : +- HashAggregate - : : +- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- Filter - : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.customer + : : +- CometExchange + : : +- CometHashAggregate + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometNativeColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.customer : +- BroadcastExchange : +- CometNativeColumnarToRow : +- CometHashAggregate - : +- CometColumnarExchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Filter - : : : +- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : +- ReusedSubquery - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- CometNativeColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.customer + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometNativeColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.customer +- BroadcastExchange +- CometNativeColumnarToRow +- CometHashAggregate - +- CometColumnarExchange - +- HashAggregate - +- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- Filter - : : +- ColumnarToRow - : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : +- ReusedSubquery - : +- BroadcastExchange - : +- CometNativeColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.date_dim - +- BroadcastExchange - +- CometNativeColumnarToRow - +- CometProject - +- CometFilter - +- CometNativeScan parquet spark_catalog.default.customer + +- CometExchange + +- CometHashAggregate + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.web_sales + : : +- SubqueryBroadcast + : : +- BroadcastExchange + : : +- CometNativeColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.date_dim + +- CometBroadcastExchange + +- CometProject + +- CometFilter + +- CometNativeScan parquet spark_catalog.default.customer -Comet accelerated 28 out of 66 eligible operators (42%). Final plan contains 14 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 61 out of 74 eligible operators (82%). Final plan contains 7 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q89.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q89.native_datafusion/extended.txt index 5c7a26057d..936cde052f 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q89.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q89.native_datafusion/extended.txt @@ -4,39 +4,34 @@ TakeOrderedAndProject +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] +- CometNativeColumnarToRow +- CometSort - +- CometColumnarExchange - +- HashAggregate - +- CometNativeColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.item - : : +- BroadcastExchange - : : +- Filter - : : +- ColumnarToRow - : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- CometNativeColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.date_dim - +- BroadcastExchange - +- CometNativeColumnarToRow - +- CometFilter - +- CometNativeScan parquet spark_catalog.default.store + +- CometExchange + +- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.item + : : +- CometBroadcastExchange + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : +- SubqueryBroadcast + : : +- BroadcastExchange + : : +- CometNativeColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.date_dim + +- CometBroadcastExchange + +- CometFilter + +- CometNativeScan parquet spark_catalog.default.store -Comet accelerated 14 out of 33 eligible operators (42%). Final plan contains 7 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 27 out of 33 eligible operators (81%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q91.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q91.native_datafusion/extended.txt index 6f982615ae..28bc6eab2f 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q91.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q91.native_datafusion/extended.txt @@ -1,59 +1,51 @@ CometNativeColumnarToRow +- CometSort - +- CometColumnarExchange - +- HashAggregate - +- CometNativeColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- Project - : : : : +- BroadcastHashJoin - : : : : :- Project - : : : : : +- BroadcastHashJoin - : : : : : :- CometNativeColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.call_center - : : : : : +- BroadcastExchange - : : : : : +- Filter - : : : : : +- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.catalog_returns [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.customer - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.customer_address - : +- BroadcastExchange - : +- CometNativeColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.customer_demographics - +- BroadcastExchange - +- CometNativeColumnarToRow - +- CometProject - +- CometFilter - +- CometNativeScan parquet spark_catalog.default.household_demographics + +- CometExchange + +- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometProject + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.call_center + : : : : : +- CometBroadcastExchange + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_returns + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometNativeColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.customer + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.customer_address + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.customer_demographics + +- CometBroadcastExchange + +- CometProject + +- CometFilter + +- CometNativeScan parquet spark_catalog.default.household_demographics -Comet accelerated 23 out of 47 eligible operators (48%). Final plan contains 10 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 45 out of 47 eligible operators (95%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q92.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q92.native_datafusion/extended.txt index b3ca85cdaf..6c5b57b3a1 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q92.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q92.native_datafusion/extended.txt @@ -1,48 +1,47 @@ -HashAggregate -+- CometNativeColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Filter - : : : +- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.item - : +- BroadcastExchange - : +- Filter - : +- HashAggregate - : +- CometNativeColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- Filter - : : +- ColumnarToRow - : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : +- ReusedSubquery - : +- BroadcastExchange - : +- CometNativeColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.date_dim - +- BroadcastExchange - +- CometNativeColumnarToRow - +- CometProject - +- CometFilter - +- CometNativeScan parquet spark_catalog.default.date_dim +CometNativeColumnarToRow ++- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.web_sales + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometNativeColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.item + : +- CometBroadcastExchange + : +- CometFilter + : +- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.web_sales + : : +- SubqueryBroadcast + : : +- BroadcastExchange + : : +- CometNativeColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.date_dim + +- CometBroadcastExchange + +- CometProject + +- CometFilter + +- CometNativeScan parquet spark_catalog.default.date_dim -Comet accelerated 14 out of 38 eligible operators (36%). Final plan contains 8 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 38 out of 42 eligible operators (90%). Final plan contains 3 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q97.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q97.native_datafusion/extended.txt index e3b777a05a..b63930a880 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q97.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q97.native_datafusion/extended.txt @@ -6,36 +6,37 @@ CometNativeColumnarToRow +- CometSortMergeJoin :- CometSort : +- CometHashAggregate - : +- CometColumnarExchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- ColumnarToRow - : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- CometNativeColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometNativeScan parquet spark_catalog.default.store_sales + : : +- SubqueryBroadcast + : : +- BroadcastExchange + : : +- CometNativeColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.date_dim +- CometSort +- CometHashAggregate - +- CometColumnarExchange - +- HashAggregate - +- Project - +- BroadcastHashJoin - :- ColumnarToRow - : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : +- ReusedSubquery - +- BroadcastExchange - +- CometNativeColumnarToRow - +- CometProject - +- CometFilter - +- CometNativeScan parquet spark_catalog.default.date_dim + +- CometExchange + +- CometHashAggregate + +- CometProject + +- CometBroadcastHashJoin + :- CometNativeScan parquet spark_catalog.default.catalog_sales + : +- SubqueryBroadcast + : +- BroadcastExchange + : +- CometNativeColumnarToRow + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.date_dim + +- CometBroadcastExchange + +- CometProject + +- CometFilter + +- CometNativeScan parquet spark_catalog.default.date_dim -Comet accelerated 20 out of 33 eligible operators (60%). Final plan contains 6 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 33 out of 37 eligible operators (89%). Final plan contains 3 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q98.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q98.native_datafusion/extended.txt index af8cf29205..6cc173560b 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q98.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q98.native_datafusion/extended.txt @@ -6,33 +6,29 @@ CometNativeColumnarToRow +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] +- CometNativeColumnarToRow +- CometSort - +- CometColumnarExchange - +- HashAggregate - +- CometNativeColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- Filter - : : +- ColumnarToRow - : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- CometNativeColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.item - +- BroadcastExchange - +- CometNativeColumnarToRow - +- CometProject - +- CometFilter - +- CometNativeScan parquet spark_catalog.default.date_dim + +- CometExchange + +- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : +- SubqueryBroadcast + : : +- BroadcastExchange + : : +- CometNativeColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.item + +- CometBroadcastExchange + +- CometProject + +- CometFilter + +- CometNativeScan parquet spark_catalog.default.date_dim -Comet accelerated 15 out of 29 eligible operators (51%). Final plan contains 7 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 25 out of 29 eligible operators (86%). Final plan contains 3 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q1.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q1.native_datafusion/extended.txt index 91b07a7538..935c103614 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q1.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q1.native_datafusion/extended.txt @@ -1,61 +1,58 @@ -TakeOrderedAndProject -+- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Filter - : : : +- HashAggregate - : : : +- CometNativeColumnarToRow - : : : +- CometColumnarExchange - : : : +- HashAggregate - : : : +- Project - : : : +- BroadcastHashJoin - : : : :- Filter - : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.store_returns [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- Filter - : : +- HashAggregate - : : +- CometNativeColumnarToRow - : : +- CometColumnarExchange - : : +- HashAggregate - : : +- HashAggregate - : : +- CometNativeColumnarToRow - : : +- CometColumnarExchange - : : +- HashAggregate - : : +- Project - : : +- BroadcastHashJoin - : : :- Filter - : : : +- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.store_returns [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : +- ReusedSubquery - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- CometNativeColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.store - +- BroadcastExchange - +- CometNativeColumnarToRow +CometNativeColumnarToRow ++- CometTakeOrderedAndProject + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometHashAggregate + : : : +- CometExchange + : : : +- CometHashAggregate + : : : +- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.store_returns + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometNativeColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometFilter + : : +- CometHashAggregate + : : +- CometExchange + : : +- CometHashAggregate + : : +- CometHashAggregate + : : +- CometExchange + : : +- CometHashAggregate + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.store_returns + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometNativeColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.store + +- CometBroadcastExchange +- CometProject +- CometFilter +- CometNativeScan parquet spark_catalog.default.customer -Comet accelerated 18 out of 49 eligible operators (36%). Final plan contains 10 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 49 out of 53 eligible operators (92%). Final plan contains 3 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q10.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q10.native_datafusion/extended.txt index 40a8ef4a6e..2785c3d0b1 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q10.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q10.native_datafusion/extended.txt @@ -10,46 +10,52 @@ TakeOrderedAndProject : :- Project : : +- Filter : : +- BroadcastHashJoin - : : :- BroadcastHashJoin - : : : :- BroadcastHashJoin - : : : : :- CometNativeColumnarToRow - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.customer - : : : : +- BroadcastExchange - : : : : +- Project - : : : : +- BroadcastHashJoin - : : : : :- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow + : : :- BroadcastHashJoin [COMET: Unsupported join type ExistenceJoin(exists#1)] + : : : :- CometNativeColumnarToRow + : : : : +- CometBroadcastHashJoin + : : : : :- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.customer + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometNativeScan parquet spark_catalog.default.store_sales + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometNativeColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange : : : : +- CometProject : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- BroadcastExchange - : : : +- Project - : : : +- BroadcastHashJoin - : : : :- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- ReusedSubquery - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow + : : : +- CometNativeColumnarToRow + : : : +- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometNativeScan parquet spark_catalog.default.web_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometNativeColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- BroadcastExchange - : : +- Project - : : +- BroadcastHashJoin - : : :- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : +- ReusedSubquery - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow + : : +- CometNativeColumnarToRow + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometNativeColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange : : +- CometProject : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -64,4 +70,4 @@ TakeOrderedAndProject +- CometFilter +- CometNativeScan parquet spark_catalog.default.customer_demographics -Comet accelerated 21 out of 54 eligible operators (38%). Final plan contains 11 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 41 out of 62 eligible operators (66%). Final plan contains 9 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q11.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q11.native_datafusion/extended.txt index 47aa9517cf..6b081dd35b 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q11.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q11.native_datafusion/extended.txt @@ -1,106 +1,99 @@ -TakeOrderedAndProject -+- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Filter - : : : +- HashAggregate - : : : +- CometNativeColumnarToRow - : : : +- CometColumnarExchange - : : : +- HashAggregate - : : : +- Project - : : : +- BroadcastHashJoin - : : : :- Project - : : : : +- BroadcastHashJoin - : : : : :- CometNativeColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.customer - : : : : +- BroadcastExchange - : : : : +- Filter - : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- HashAggregate - : : +- CometNativeColumnarToRow - : : +- CometColumnarExchange - : : +- HashAggregate - : : +- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.customer - : : : +- BroadcastExchange - : : : +- Filter - : : : +- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- Filter - : +- HashAggregate - : +- CometNativeColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.customer - : : +- BroadcastExchange - : : +- Filter - : : +- ColumnarToRow - : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : +- ReusedSubquery - : +- BroadcastExchange - : +- CometNativeColumnarToRow - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.date_dim - +- BroadcastExchange - +- HashAggregate - +- CometNativeColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- CometNativeColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.customer - : +- BroadcastExchange - : +- Filter - : +- ColumnarToRow - : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : +- ReusedSubquery - +- BroadcastExchange - +- CometNativeColumnarToRow - +- CometFilter - +- CometNativeScan parquet spark_catalog.default.date_dim +CometNativeColumnarToRow ++- CometTakeOrderedAndProject + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometHashAggregate + : : : +- CometExchange + : : : +- CometHashAggregate + : : : +- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.customer + : : : : +- CometBroadcastExchange + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometNativeColumnarToRow + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometHashAggregate + : : +- CometExchange + : : +- CometHashAggregate + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.customer + : : : +- CometBroadcastExchange + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometNativeColumnarToRow + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometFilter + : +- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.customer + : : +- CometBroadcastExchange + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.web_sales + : : +- SubqueryBroadcast + : : +- BroadcastExchange + : : +- CometNativeColumnarToRow + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.date_dim + +- CometBroadcastExchange + +- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.customer + : +- CometBroadcastExchange + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.web_sales + : +- SubqueryBroadcast + : +- BroadcastExchange + : +- CometNativeColumnarToRow + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.date_dim + +- CometBroadcastExchange + +- CometFilter + +- CometNativeScan parquet spark_catalog.default.date_dim -Comet accelerated 28 out of 86 eligible operators (32%). Final plan contains 18 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 84 out of 92 eligible operators (91%). Final plan contains 5 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q12.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q12.native_datafusion/extended.txt index 15648408fc..a8c2531a38 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q12.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q12.native_datafusion/extended.txt @@ -3,33 +3,29 @@ TakeOrderedAndProject +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] +- CometNativeColumnarToRow +- CometSort - +- CometColumnarExchange - +- HashAggregate - +- CometNativeColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- Filter - : : +- ColumnarToRow - : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- CometNativeColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.item - +- BroadcastExchange - +- CometNativeColumnarToRow - +- CometProject - +- CometFilter - +- CometNativeScan parquet spark_catalog.default.date_dim + +- CometExchange + +- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.web_sales + : : +- SubqueryBroadcast + : : +- BroadcastExchange + : : +- CometNativeColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.item + +- CometBroadcastExchange + +- CometProject + +- CometFilter + +- CometNativeScan parquet spark_catalog.default.date_dim -Comet accelerated 12 out of 27 eligible operators (44%). Final plan contains 6 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 22 out of 27 eligible operators (81%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q13.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q13.native_datafusion/extended.txt index 9ff51aab85..42ea1c4966 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q13.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q13.native_datafusion/extended.txt @@ -1,48 +1,42 @@ -HashAggregate -+- CometNativeColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- Project - : : : : +- BroadcastHashJoin - : : : : :- Filter - : : : : : +- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.store - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.customer_address - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- CometNativeColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.customer_demographics - +- BroadcastExchange - +- CometNativeColumnarToRow - +- CometFilter - +- CometNativeScan parquet spark_catalog.default.household_demographics +CometNativeColumnarToRow ++- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometNativeColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.store + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.customer_address + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.customer_demographics + +- CometBroadcastExchange + +- CometFilter + +- CometNativeScan parquet spark_catalog.default.household_demographics -Comet accelerated 17 out of 38 eligible operators (44%). Final plan contains 8 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 36 out of 38 eligible operators (94%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14a.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14a.native_datafusion/extended.txt index 223593e3d0..0dcd47f57f 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14a.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14a.native_datafusion/extended.txt @@ -1,553 +1,554 @@ -TakeOrderedAndProject -+- HashAggregate - +- CometNativeColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Expand - +- Union - :- Project - : +- Filter +CometNativeColumnarToRow ++- CometTakeOrderedAndProject + +- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometExpand + +- CometUnion + :- CometProject + : +- CometFilter : : +- Subquery - : : +- HashAggregate - : : +- CometNativeColumnarToRow - : : +- CometColumnarExchange - : : +- HashAggregate - : : +- Union - : : :- Project - : : : +- BroadcastHashJoin - : : : :- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- ReusedSubquery - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : :- Project - : : : +- BroadcastHashJoin - : : : :- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- ReusedSubquery - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- Project - : : +- BroadcastHashJoin - : : :- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : +- ReusedSubquery - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- HashAggregate - : +- CometNativeColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- BroadcastHashJoin - : : : :- Filter - : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- Project - : : : +- BroadcastHashJoin - : : : :- CometNativeColumnarToRow - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : +- BroadcastExchange - : : : +- BroadcastHashJoin - : : : :- CometNativeColumnarToRow - : : : : +- CometHashAggregate - : : : : +- CometColumnarExchange - : : : : +- HashAggregate - : : : : +- Project - : : : : +- BroadcastHashJoin - : : : : :- Project - : : : : : +- BroadcastHashJoin - : : : : : :- Filter - : : : : : : +- ColumnarToRow - : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometNativeColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : : +- BroadcastExchange - : : : : : +- BroadcastHashJoin - : : : : : :- CometNativeColumnarToRow - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : : +- BroadcastExchange - : : : : : +- Project - : : : : : +- BroadcastHashJoin - : : : : : :- Project - : : : : : : +- BroadcastHashJoin - : : : : : : :- Filter - : : : : : : : +- ColumnarToRow - : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : : : +- ReusedSubquery - : : : : : : +- BroadcastExchange - : : : : : : +- CometNativeColumnarToRow - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : : +- BroadcastExchange - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- Project - : : : +- BroadcastHashJoin - : : : :- Project - : : : : +- BroadcastHashJoin - : : : : :- Filter - : : : : : +- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : +- ReusedSubquery - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- BroadcastHashJoin - : : :- CometNativeColumnarToRow - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.item - : : +- BroadcastExchange - : : +- Project - : : +- BroadcastHashJoin - : : :- CometNativeColumnarToRow - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.item - : : +- BroadcastExchange - : : +- BroadcastHashJoin - : : :- CometNativeColumnarToRow - : : : +- CometHashAggregate - : : : +- CometColumnarExchange - : : : +- HashAggregate - : : : +- Project - : : : +- BroadcastHashJoin - : : : :- Project - : : : : +- BroadcastHashJoin - : : : : :- Filter - : : : : : +- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : +- BroadcastExchange - : : : : +- BroadcastHashJoin - : : : : :- CometNativeColumnarToRow - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : +- BroadcastExchange - : : : : +- Project - : : : : +- BroadcastHashJoin - : : : : :- Project - : : : : : +- BroadcastHashJoin - : : : : : :- Filter - : : : : : : +- ColumnarToRow - : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : : +- ReusedSubquery - : : : : : +- BroadcastExchange - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- Filter - : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- ReusedSubquery - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.item - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- CometNativeColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.date_dim - :- Project - : +- Filter + : : +- CometNativeColumnarToRow + : : +- CometHashAggregate + : : +- CometExchange + : : +- CometHashAggregate + : : +- CometUnion + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometNativeScan parquet spark_catalog.default.store_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometNativeColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometNativeColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometNativeScan parquet spark_catalog.default.web_sales + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometNativeColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometNativeColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : +- CometBroadcastExchange + : : : +- CometBroadcastHashJoin + : : : :- CometHashAggregate + : : : : +- CometExchange + : : : : +- CometHashAggregate + : : : : +- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometProject + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : : : +- SubqueryBroadcast + : : : : : : +- BroadcastExchange + : : : : : : +- CometNativeColumnarToRow + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- CometBroadcastExchange + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : : : +- CometBroadcastExchange + : : : : : +- CometProject + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometProject + : : : : : : +- CometBroadcastHashJoin + : : : : : : :- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : : : : +- SubqueryBroadcast + : : : : : : : +- BroadcastExchange + : : : : : : : +- CometNativeColumnarToRow + : : : : : : : +- CometProject + : : : : : : : +- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : +- CometBroadcastExchange + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : : : +- CometBroadcastExchange + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.web_sales + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometNativeColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.item + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.item + : : +- CometBroadcastExchange + : : +- CometBroadcastHashJoin + : : :- CometHashAggregate + : : : +- CometExchange + : : : +- CometHashAggregate + : : : +- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometNativeColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometBroadcastHashJoin + : : : : :- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometProject + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : : : +- SubqueryBroadcast + : : : : : : +- BroadcastExchange + : : : : : : +- CometNativeColumnarToRow + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- CometBroadcastExchange + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.web_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometNativeColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.item + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.date_dim + :- CometProject + : +- CometFilter : : +- ReusedSubquery - : +- HashAggregate - : +- CometNativeColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- BroadcastHashJoin - : : : :- Filter - : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- ReusedSubquery - : : : +- BroadcastExchange - : : : +- Project - : : : +- BroadcastHashJoin - : : : :- CometNativeColumnarToRow - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : +- BroadcastExchange - : : : +- BroadcastHashJoin - : : : :- CometNativeColumnarToRow - : : : : +- CometHashAggregate - : : : : +- CometColumnarExchange - : : : : +- HashAggregate - : : : : +- Project - : : : : +- BroadcastHashJoin - : : : : :- Project - : : : : : +- BroadcastHashJoin - : : : : : :- Filter - : : : : : : +- ColumnarToRow - : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometNativeColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : : +- BroadcastExchange - : : : : : +- BroadcastHashJoin - : : : : : :- CometNativeColumnarToRow - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : : +- BroadcastExchange - : : : : : +- Project - : : : : : +- BroadcastHashJoin - : : : : : :- Project - : : : : : : +- BroadcastHashJoin - : : : : : : :- Filter - : : : : : : : +- ColumnarToRow - : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : : : +- ReusedSubquery - : : : : : : +- BroadcastExchange - : : : : : : +- CometNativeColumnarToRow - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : : +- BroadcastExchange - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- Project - : : : +- BroadcastHashJoin - : : : :- Project - : : : : +- BroadcastHashJoin - : : : : :- Filter - : : : : : +- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : +- ReusedSubquery - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- BroadcastHashJoin - : : :- CometNativeColumnarToRow - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.item - : : +- BroadcastExchange - : : +- Project - : : +- BroadcastHashJoin - : : :- CometNativeColumnarToRow - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.item - : : +- BroadcastExchange - : : +- BroadcastHashJoin - : : :- CometNativeColumnarToRow - : : : +- CometHashAggregate - : : : +- CometColumnarExchange - : : : +- HashAggregate - : : : +- Project - : : : +- BroadcastHashJoin - : : : :- Project - : : : : +- BroadcastHashJoin - : : : : :- Filter - : : : : : +- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : +- BroadcastExchange - : : : : +- BroadcastHashJoin - : : : : :- CometNativeColumnarToRow - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : +- BroadcastExchange - : : : : +- Project - : : : : +- BroadcastHashJoin - : : : : :- Project - : : : : : +- BroadcastHashJoin - : : : : : :- Filter - : : : : : : +- ColumnarToRow - : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : : +- ReusedSubquery - : : : : : +- BroadcastExchange - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- Filter - : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- ReusedSubquery - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.item - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- CometNativeColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.date_dim - +- Project - +- Filter + : +- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometNativeColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : +- CometBroadcastExchange + : : : +- CometBroadcastHashJoin + : : : :- CometHashAggregate + : : : : +- CometExchange + : : : : +- CometHashAggregate + : : : : +- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometProject + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : : : +- SubqueryBroadcast + : : : : : : +- BroadcastExchange + : : : : : : +- CometNativeColumnarToRow + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- CometBroadcastExchange + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : : : +- CometBroadcastExchange + : : : : : +- CometProject + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometProject + : : : : : : +- CometBroadcastHashJoin + : : : : : : :- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : : : : +- SubqueryBroadcast + : : : : : : : +- BroadcastExchange + : : : : : : : +- CometNativeColumnarToRow + : : : : : : : +- CometProject + : : : : : : : +- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : +- CometBroadcastExchange + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : : : +- CometBroadcastExchange + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.web_sales + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometNativeColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.item + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.item + : : +- CometBroadcastExchange + : : +- CometBroadcastHashJoin + : : :- CometHashAggregate + : : : +- CometExchange + : : : +- CometHashAggregate + : : : +- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometNativeColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometBroadcastHashJoin + : : : : :- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometProject + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : : : +- SubqueryBroadcast + : : : : : : +- BroadcastExchange + : : : : : : +- CometNativeColumnarToRow + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- CometBroadcastExchange + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.web_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometNativeColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.item + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.date_dim + +- CometProject + +- CometFilter : +- ReusedSubquery - +- HashAggregate - +- CometNativeColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- BroadcastHashJoin - : : :- Filter - : : : +- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : +- ReusedSubquery - : : +- BroadcastExchange - : : +- Project - : : +- BroadcastHashJoin - : : :- CometNativeColumnarToRow - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.item - : : +- BroadcastExchange - : : +- BroadcastHashJoin - : : :- CometNativeColumnarToRow - : : : +- CometHashAggregate - : : : +- CometColumnarExchange - : : : +- HashAggregate - : : : +- Project - : : : +- BroadcastHashJoin - : : : :- Project - : : : : +- BroadcastHashJoin - : : : : :- Filter - : : : : : +- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : +- BroadcastExchange - : : : : +- BroadcastHashJoin - : : : : :- CometNativeColumnarToRow - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : +- BroadcastExchange - : : : : +- Project - : : : : +- BroadcastHashJoin - : : : : :- Project - : : : : : +- BroadcastHashJoin - : : : : : :- Filter - : : : : : : +- ColumnarToRow - : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : : +- ReusedSubquery - : : : : : +- BroadcastExchange - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- Filter - : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- ReusedSubquery - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.item - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- BroadcastHashJoin - : :- CometNativeColumnarToRow - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.item - : +- BroadcastExchange - : +- Project - : +- BroadcastHashJoin - : :- CometNativeColumnarToRow - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.item - : +- BroadcastExchange - : +- BroadcastHashJoin - : :- CometNativeColumnarToRow - : : +- CometHashAggregate - : : +- CometColumnarExchange - : : +- HashAggregate - : : +- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- Filter - : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- BroadcastHashJoin - : : : :- CometNativeColumnarToRow - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : +- BroadcastExchange - : : : +- Project - : : : +- BroadcastHashJoin - : : : :- Project - : : : : +- BroadcastHashJoin - : : : : :- Filter - : : : : : +- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : +- ReusedSubquery - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Filter - : : : +- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : +- ReusedSubquery - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.item - : +- BroadcastExchange - : +- CometNativeColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.date_dim - +- BroadcastExchange - +- CometNativeColumnarToRow - +- CometProject - +- CometFilter - +- CometNativeScan parquet spark_catalog.default.date_dim + +- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.web_sales + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometNativeColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.item + : : +- CometBroadcastExchange + : : +- CometBroadcastHashJoin + : : :- CometHashAggregate + : : : +- CometExchange + : : : +- CometHashAggregate + : : : +- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometNativeColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometBroadcastHashJoin + : : : : :- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometProject + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : : : +- SubqueryBroadcast + : : : : : : +- BroadcastExchange + : : : : : : +- CometNativeColumnarToRow + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- CometBroadcastExchange + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.web_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometNativeColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.item + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometBroadcastHashJoin + : :- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.item + : +- CometBroadcastExchange + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.item + : +- CometBroadcastExchange + : +- CometBroadcastHashJoin + : :- CometHashAggregate + : : +- CometExchange + : : +- CometHashAggregate + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometNativeColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometNativeColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.web_sales + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometNativeColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.item + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.date_dim + +- CometBroadcastExchange + +- CometProject + +- CometFilter + +- CometNativeScan parquet spark_catalog.default.date_dim -Comet accelerated 164 out of 458 eligible operators (35%). Final plan contains 93 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 475 out of 526 eligible operators (90%). Final plan contains 26 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14b.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14b.native_datafusion/extended.txt index 7921f2fa61..50139381f2 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14b.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14b.native_datafusion/extended.txt @@ -1,404 +1,394 @@ -TakeOrderedAndProject -+- BroadcastHashJoin - :- Filter - : : +- Subquery - : : +- HashAggregate - : : +- CometNativeColumnarToRow - : : +- CometColumnarExchange - : : +- HashAggregate - : : +- Union - : : :- Project - : : : +- BroadcastHashJoin - : : : :- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- ReusedSubquery - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : :- Project - : : : +- BroadcastHashJoin - : : : :- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- ReusedSubquery - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- Project - : : +- BroadcastHashJoin - : : :- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : +- ReusedSubquery - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- HashAggregate - : +- CometNativeColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- BroadcastHashJoin - : : : :- Filter - : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : : +- Subquery - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- Project - : : : +- BroadcastHashJoin - : : : :- CometNativeColumnarToRow - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : +- BroadcastExchange - : : : +- BroadcastHashJoin - : : : :- CometNativeColumnarToRow - : : : : +- CometHashAggregate - : : : : +- CometColumnarExchange - : : : : +- HashAggregate - : : : : +- Project - : : : : +- BroadcastHashJoin - : : : : :- Project - : : : : : +- BroadcastHashJoin - : : : : : :- Filter - : : : : : : +- ColumnarToRow - : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometNativeColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : : +- BroadcastExchange - : : : : : +- BroadcastHashJoin - : : : : : :- CometNativeColumnarToRow - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : : +- BroadcastExchange - : : : : : +- Project - : : : : : +- BroadcastHashJoin - : : : : : :- Project - : : : : : : +- BroadcastHashJoin - : : : : : : :- Filter - : : : : : : : +- ColumnarToRow - : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : : : +- ReusedSubquery - : : : : : : +- BroadcastExchange - : : : : : : +- CometNativeColumnarToRow - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : : +- BroadcastExchange - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- Project - : : : +- BroadcastHashJoin - : : : :- Project - : : : : +- BroadcastHashJoin - : : : : :- Filter - : : : : : +- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : +- ReusedSubquery - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- BroadcastHashJoin - : : :- CometNativeColumnarToRow - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.item - : : +- BroadcastExchange - : : +- Project - : : +- BroadcastHashJoin - : : :- CometNativeColumnarToRow - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.item - : : +- BroadcastExchange - : : +- BroadcastHashJoin - : : :- CometNativeColumnarToRow - : : : +- CometHashAggregate - : : : +- CometColumnarExchange - : : : +- HashAggregate - : : : +- Project - : : : +- BroadcastHashJoin - : : : :- Project - : : : : +- BroadcastHashJoin - : : : : :- Filter - : : : : : +- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : +- BroadcastExchange - : : : : +- BroadcastHashJoin - : : : : :- CometNativeColumnarToRow - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : +- BroadcastExchange - : : : : +- Project - : : : : +- BroadcastHashJoin - : : : : :- Project - : : : : : +- BroadcastHashJoin - : : : : : :- Filter - : : : : : : +- ColumnarToRow - : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : : +- ReusedSubquery - : : : : : +- BroadcastExchange - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- Filter - : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- ReusedSubquery - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.item - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- CometNativeColumnarToRow - : +- CometProject - : +- CometFilter - : : +- Subquery - : : +- CometNativeColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- CometNativeScan parquet spark_catalog.default.date_dim - +- BroadcastExchange - +- Filter - : +- ReusedSubquery - +- HashAggregate - +- CometNativeColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- BroadcastHashJoin - : : :- Filter - : : : +- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : : +- Subquery - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- Project - : : +- BroadcastHashJoin - : : :- CometNativeColumnarToRow - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.item - : : +- BroadcastExchange - : : +- BroadcastHashJoin - : : :- CometNativeColumnarToRow - : : : +- CometHashAggregate - : : : +- CometColumnarExchange - : : : +- HashAggregate - : : : +- Project - : : : +- BroadcastHashJoin - : : : :- Project - : : : : +- BroadcastHashJoin - : : : : :- Filter - : : : : : +- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : +- BroadcastExchange - : : : : +- BroadcastHashJoin - : : : : :- CometNativeColumnarToRow - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : +- BroadcastExchange - : : : : +- Project - : : : : +- BroadcastHashJoin - : : : : :- Project - : : : : : +- BroadcastHashJoin - : : : : : :- Filter - : : : : : : +- ColumnarToRow - : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : : +- ReusedSubquery - : : : : : +- BroadcastExchange - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- Filter - : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- ReusedSubquery - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.item - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- BroadcastHashJoin - : :- CometNativeColumnarToRow - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.item - : +- BroadcastExchange - : +- Project - : +- BroadcastHashJoin - : :- CometNativeColumnarToRow - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.item - : +- BroadcastExchange - : +- BroadcastHashJoin - : :- CometNativeColumnarToRow - : : +- CometHashAggregate - : : +- CometColumnarExchange - : : +- HashAggregate - : : +- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- Filter - : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- BroadcastHashJoin - : : : :- CometNativeColumnarToRow - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : +- BroadcastExchange - : : : +- Project - : : : +- BroadcastHashJoin - : : : :- Project - : : : : +- BroadcastHashJoin - : : : : :- Filter - : : : : : +- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : +- ReusedSubquery - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Filter - : : : +- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : +- ReusedSubquery - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.item - : +- BroadcastExchange - : +- CometNativeColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.date_dim - +- BroadcastExchange - +- CometNativeColumnarToRow - +- CometProject - +- CometFilter - : +- Subquery - : +- CometNativeColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.date_dim - +- CometNativeScan parquet spark_catalog.default.date_dim +CometNativeColumnarToRow ++- CometTakeOrderedAndProject + +- CometBroadcastHashJoin + :- CometFilter + : : +- Subquery + : : +- CometNativeColumnarToRow + : : +- CometHashAggregate + : : +- CometExchange + : : +- CometHashAggregate + : : +- CometUnion + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometNativeScan parquet spark_catalog.default.store_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometNativeColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometNativeColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometNativeScan parquet spark_catalog.default.web_sales + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometNativeColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometNativeColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : : +- Subquery + : : : : : +- CometNativeColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : +- CometBroadcastExchange + : : : +- CometBroadcastHashJoin + : : : :- CometHashAggregate + : : : : +- CometExchange + : : : : +- CometHashAggregate + : : : : +- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometProject + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : : : +- SubqueryBroadcast + : : : : : : +- BroadcastExchange + : : : : : : +- CometNativeColumnarToRow + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- CometBroadcastExchange + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : : : +- CometBroadcastExchange + : : : : : +- CometProject + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometProject + : : : : : : +- CometBroadcastHashJoin + : : : : : : :- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : : : : +- SubqueryBroadcast + : : : : : : : +- BroadcastExchange + : : : : : : : +- CometNativeColumnarToRow + : : : : : : : +- CometProject + : : : : : : : +- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : +- CometBroadcastExchange + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : : : +- CometBroadcastExchange + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.web_sales + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometNativeColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.item + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.item + : : +- CometBroadcastExchange + : : +- CometBroadcastHashJoin + : : :- CometHashAggregate + : : : +- CometExchange + : : : +- CometHashAggregate + : : : +- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometNativeColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometBroadcastHashJoin + : : : : :- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometProject + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : : : +- SubqueryBroadcast + : : : : : : +- BroadcastExchange + : : : : : : +- CometNativeColumnarToRow + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- CometBroadcastExchange + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.web_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometNativeColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.item + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : : +- ReusedSubquery + : +- CometNativeScan parquet spark_catalog.default.date_dim + +- CometBroadcastExchange + +- CometFilter + : +- ReusedSubquery + +- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometNativeColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : : +- Subquery + : : : : +- CometNativeColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.item + : : +- CometBroadcastExchange + : : +- CometBroadcastHashJoin + : : :- CometHashAggregate + : : : +- CometExchange + : : : +- CometHashAggregate + : : : +- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometNativeColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometBroadcastHashJoin + : : : : :- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometProject + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : : : +- SubqueryBroadcast + : : : : : : +- BroadcastExchange + : : : : : : +- CometNativeColumnarToRow + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- CometBroadcastExchange + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.web_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometNativeColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.item + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometBroadcastHashJoin + : :- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.item + : +- CometBroadcastExchange + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.item + : +- CometBroadcastExchange + : +- CometBroadcastHashJoin + : :- CometHashAggregate + : : +- CometExchange + : : +- CometHashAggregate + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometNativeColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometNativeColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.web_sales + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometNativeColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.item + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.date_dim + +- CometBroadcastExchange + +- CometProject + +- CometFilter + : +- ReusedSubquery + +- CometNativeScan parquet spark_catalog.default.date_dim -Comet accelerated 128 out of 333 eligible operators (38%). Final plan contains 69 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 331 out of 371 eligible operators (89%). Final plan contains 21 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q15.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q15.native_datafusion/extended.txt index d0d170db53..42432bdc12 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q15.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q15.native_datafusion/extended.txt @@ -1,36 +1,32 @@ -TakeOrderedAndProject -+- HashAggregate - +- CometNativeColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Filter - : : : +- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.customer - : +- BroadcastExchange - : +- CometNativeColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.customer_address - +- BroadcastExchange - +- CometNativeColumnarToRow - +- CometProject - +- CometFilter - +- CometNativeScan parquet spark_catalog.default.date_dim +CometNativeColumnarToRow ++- CometTakeOrderedAndProject + +- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometNativeColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.customer + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.customer_address + +- CometBroadcastExchange + +- CometProject + +- CometFilter + +- CometNativeScan parquet spark_catalog.default.date_dim -Comet accelerated 12 out of 28 eligible operators (42%). Final plan contains 6 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 26 out of 28 eligible operators (92%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q17.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q17.native_datafusion/extended.txt index 15c6bec5dd..f58c012650 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q17.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q17.native_datafusion/extended.txt @@ -1,70 +1,67 @@ -TakeOrderedAndProject -+- HashAggregate - +- CometNativeColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- Project - : : : : +- BroadcastHashJoin - : : : : :- Project - : : : : : +- BroadcastHashJoin - : : : : : :- Project - : : : : : : +- BroadcastHashJoin - : : : : : : :- Filter - : : : : : : : +- ColumnarToRow - : : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : : : +- SubqueryBroadcast - : : : : : : : +- BroadcastExchange - : : : : : : : +- CometNativeColumnarToRow - : : : : : : : +- CometProject - : : : : : : : +- CometFilter - : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : : : +- BroadcastExchange - : : : : : : +- Filter - : : : : : : +- ColumnarToRow - : : : : : : +- Scan parquet spark_catalog.default.store_returns [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometNativeColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : : +- BroadcastExchange - : : : : : +- Filter - : : : : : +- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : +- ReusedSubquery - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- CometNativeColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.store - +- BroadcastExchange - +- CometNativeColumnarToRow - +- CometProject - +- CometFilter - +- CometNativeScan parquet spark_catalog.default.item +CometNativeColumnarToRow ++- CometTakeOrderedAndProject + +- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometProject + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometProject + : : : : : : +- CometBroadcastHashJoin + : : : : : : :- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : : : : +- SubqueryBroadcast + : : : : : : : +- BroadcastExchange + : : : : : : : +- CometNativeColumnarToRow + : : : : : : : +- CometProject + : : : : : : : +- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : +- CometBroadcastExchange + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.store_returns + : : : : : : +- SubqueryBroadcast + : : : : : : +- BroadcastExchange + : : : : : : +- CometNativeColumnarToRow + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- CometBroadcastExchange + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometNativeColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.store + +- CometBroadcastExchange + +- CometProject + +- CometFilter + +- CometNativeScan parquet spark_catalog.default.item -Comet accelerated 22 out of 57 eligible operators (38%). Final plan contains 11 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 55 out of 61 eligible operators (90%). Final plan contains 4 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q18.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q18.native_datafusion/extended.txt index 53fa7b6342..b3485a3421 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q18.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q18.native_datafusion/extended.txt @@ -1,58 +1,51 @@ -TakeOrderedAndProject -+- HashAggregate - +- CometNativeColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Expand - +- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- Project - : : : : +- BroadcastHashJoin - : : : : :- Project - : : : : : +- BroadcastHashJoin - : : : : : :- Filter - : : : : : : +- ColumnarToRow - : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometNativeColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : : +- BroadcastExchange - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.customer_demographics - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.customer - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.customer_demographics - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.customer_address - : +- BroadcastExchange - : +- CometNativeColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.date_dim - +- BroadcastExchange - +- CometNativeColumnarToRow - +- CometProject - +- CometFilter - +- CometNativeScan parquet spark_catalog.default.item +CometNativeColumnarToRow ++- CometTakeOrderedAndProject + +- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometExpand + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometProject + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : : : +- SubqueryBroadcast + : : : : : : +- BroadcastExchange + : : : : : : +- CometNativeColumnarToRow + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- CometBroadcastExchange + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.customer_demographics + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.customer + : : : +- CometBroadcastExchange + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.customer_demographics + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.customer_address + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.date_dim + +- CometBroadcastExchange + +- CometProject + +- CometFilter + +- CometNativeScan parquet spark_catalog.default.item -Comet accelerated 21 out of 47 eligible operators (44%). Final plan contains 9 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 45 out of 47 eligible operators (95%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q20.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q20.native_datafusion/extended.txt index 6634edcbed..c4f54f4289 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q20.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q20.native_datafusion/extended.txt @@ -3,33 +3,29 @@ TakeOrderedAndProject +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] +- CometNativeColumnarToRow +- CometSort - +- CometColumnarExchange - +- HashAggregate - +- CometNativeColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- Filter - : : +- ColumnarToRow - : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- CometNativeColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.item - +- BroadcastExchange - +- CometNativeColumnarToRow - +- CometProject - +- CometFilter - +- CometNativeScan parquet spark_catalog.default.date_dim + +- CometExchange + +- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : +- SubqueryBroadcast + : : +- BroadcastExchange + : : +- CometNativeColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.item + +- CometBroadcastExchange + +- CometProject + +- CometFilter + +- CometNativeScan parquet spark_catalog.default.date_dim -Comet accelerated 12 out of 27 eligible operators (44%). Final plan contains 6 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 22 out of 27 eligible operators (81%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q21.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q21.native_datafusion/extended.txt index 407648ea84..8522eb4ab9 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q21.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q21.native_datafusion/extended.txt @@ -1,35 +1,31 @@ -TakeOrderedAndProject -+- Filter - +- HashAggregate - +- CometNativeColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Filter - : : : +- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.inventory [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.warehouse - : +- BroadcastExchange - : +- CometNativeColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.item - +- BroadcastExchange - +- CometNativeColumnarToRow - +- CometFilter - +- CometNativeScan parquet spark_catalog.default.date_dim +CometNativeColumnarToRow ++- CometTakeOrderedAndProject + +- CometFilter + +- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.inventory + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometNativeColumnarToRow + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.warehouse + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.item + +- CometBroadcastExchange + +- CometFilter + +- CometNativeScan parquet spark_catalog.default.date_dim -Comet accelerated 10 out of 27 eligible operators (37%). Final plan contains 6 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 25 out of 27 eligible operators (92%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q22.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q22.native_datafusion/extended.txt index e1766e64d8..6b78d21629 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q22.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q22.native_datafusion/extended.txt @@ -1,37 +1,33 @@ -TakeOrderedAndProject -+- HashAggregate - +- CometNativeColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Expand - +- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Filter - : : : +- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.inventory [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- CometNativeColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.item - +- BroadcastExchange - +- CometNativeColumnarToRow - +- CometFilter - +- CometNativeScan parquet spark_catalog.default.warehouse +CometNativeColumnarToRow ++- CometTakeOrderedAndProject + +- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometExpand + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.inventory + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometNativeColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.item + +- CometBroadcastExchange + +- CometFilter + +- CometNativeScan parquet spark_catalog.default.warehouse -Comet accelerated 12 out of 29 eligible operators (41%). Final plan contains 6 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 27 out of 29 eligible operators (93%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q23a.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q23a.native_datafusion/extended.txt index 7040e78da1..8a9990d536 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q23a.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q23a.native_datafusion/extended.txt @@ -8,80 +8,71 @@ CometNativeColumnarToRow : :- CometProject : : +- CometSortMergeJoin : : :- CometSort - : : : +- CometColumnarExchange - : : : +- Project - : : : +- BroadcastHashJoin - : : : :- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- Project - : : : +- Filter - : : : +- HashAggregate - : : : +- CometNativeColumnarToRow - : : : +- CometColumnarExchange - : : : +- HashAggregate - : : : +- Project - : : : +- BroadcastHashJoin - : : : :- Project - : : : : +- BroadcastHashJoin - : : : : :- Filter - : : : : : +- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.item + : : : +- CometExchange + : : : +- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometNativeColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometHashAggregate + : : : +- CometExchange + : : : +- CometHashAggregate + : : : +- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometNativeColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.item : : +- CometSort : : +- CometProject : : +- CometFilter : : : +- Subquery - : : : +- HashAggregate - : : : +- CometNativeColumnarToRow - : : : +- CometColumnarExchange - : : : +- HashAggregate - : : : +- HashAggregate - : : : +- CometNativeColumnarToRow - : : : +- CometColumnarExchange - : : : +- HashAggregate - : : : +- Project - : : : +- BroadcastHashJoin - : : : :- Project - : : : : +- BroadcastHashJoin - : : : : :- Filter - : : : : : +- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.customer - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometNativeColumnarToRow + : : : +- CometHashAggregate + : : : +- CometExchange + : : : +- CometHashAggregate + : : : +- CometHashAggregate + : : : +- CometExchange + : : : +- CometHashAggregate + : : : +- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometNativeColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.customer + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- CometHashAggregate : : +- CometExchange : : +- CometHashAggregate @@ -102,41 +93,41 @@ CometNativeColumnarToRow :- CometProject : +- CometSortMergeJoin : :- CometSort - : : +- CometColumnarExchange - : : +- Project - : : +- BroadcastHashJoin - : : :- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : +- ReusedSubquery - : : +- BroadcastExchange - : : +- Project - : : +- Filter - : : +- HashAggregate - : : +- CometNativeColumnarToRow - : : +- CometColumnarExchange - : : +- HashAggregate - : : +- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- Filter - : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.item + : : +- CometExchange + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometNativeScan parquet spark_catalog.default.web_sales + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometNativeColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometHashAggregate + : : +- CometExchange + : : +- CometHashAggregate + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometNativeColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.item : +- CometSort : +- CometProject : +- CometFilter @@ -157,4 +148,4 @@ CometNativeColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.date_dim -Comet accelerated 83 out of 138 eligible operators (60%). Final plan contains 20 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 130 out of 142 eligible operators (91%). Final plan contains 7 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q23b.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q23b.native_datafusion/extended.txt index 188775e7df..52090c668e 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q23b.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q23b.native_datafusion/extended.txt @@ -10,81 +10,72 @@ CometNativeColumnarToRow : : +- CometBroadcastHashJoin : : :- CometSortMergeJoin : : : :- CometSort - : : : : +- CometColumnarExchange - : : : : +- Project - : : : : +- BroadcastHashJoin - : : : : :- Filter - : : : : : +- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : +- BroadcastExchange - : : : : +- Project - : : : : +- Filter - : : : : +- HashAggregate - : : : : +- CometNativeColumnarToRow - : : : : +- CometColumnarExchange - : : : : +- HashAggregate - : : : : +- Project - : : : : +- BroadcastHashJoin - : : : : :- Project - : : : : : +- BroadcastHashJoin - : : : : : :- Filter - : : : : : : +- ColumnarToRow - : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometNativeColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : : +- BroadcastExchange - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : : +- CometExchange + : : : : +- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometNativeColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometHashAggregate + : : : : +- CometExchange + : : : : +- CometHashAggregate + : : : : +- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometProject + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : : : +- SubqueryBroadcast + : : : : : : +- BroadcastExchange + : : : : : : +- CometNativeColumnarToRow + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- CometBroadcastExchange + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.item : : : +- CometSort : : : +- CometProject : : : +- CometFilter : : : : +- Subquery - : : : : +- HashAggregate - : : : : +- CometNativeColumnarToRow - : : : : +- CometColumnarExchange - : : : : +- HashAggregate - : : : : +- HashAggregate - : : : : +- CometNativeColumnarToRow - : : : : +- CometColumnarExchange - : : : : +- HashAggregate - : : : : +- Project - : : : : +- BroadcastHashJoin - : : : : :- Project - : : : : : +- BroadcastHashJoin - : : : : : :- Filter - : : : : : : +- ColumnarToRow - : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometNativeColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : : +- BroadcastExchange - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.customer - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometNativeColumnarToRow + : : : : +- CometHashAggregate + : : : : +- CometExchange + : : : : +- CometHashAggregate + : : : : +- CometHashAggregate + : : : : +- CometExchange + : : : : +- CometHashAggregate + : : : : +- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometProject + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : : : +- SubqueryBroadcast + : : : : : : +- BroadcastExchange + : : : : : : +- CometNativeColumnarToRow + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- CometBroadcastExchange + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.customer + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- CometHashAggregate : : : +- CometExchange : : : +- CometHashAggregate @@ -131,42 +122,42 @@ CometNativeColumnarToRow : +- CometBroadcastHashJoin : :- CometSortMergeJoin : : :- CometSort - : : : +- CometColumnarExchange - : : : +- Project - : : : +- BroadcastHashJoin - : : : :- Filter - : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- ReusedSubquery - : : : +- BroadcastExchange - : : : +- Project - : : : +- Filter - : : : +- HashAggregate - : : : +- CometNativeColumnarToRow - : : : +- CometColumnarExchange - : : : +- HashAggregate - : : : +- Project - : : : +- BroadcastHashJoin - : : : :- Project - : : : : +- BroadcastHashJoin - : : : : :- Filter - : : : : : +- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.item + : : : +- CometExchange + : : : +- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.web_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometNativeColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometHashAggregate + : : : +- CometExchange + : : : +- CometHashAggregate + : : : +- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometNativeColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.item : : +- CometSort : : +- CometProject : : +- CometFilter @@ -209,4 +200,4 @@ CometNativeColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.date_dim -Comet accelerated 131 out of 190 eligible operators (68%). Final plan contains 20 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 180 out of 194 eligible operators (92%). Final plan contains 7 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q25.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q25.native_datafusion/extended.txt index 15c6bec5dd..f58c012650 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q25.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q25.native_datafusion/extended.txt @@ -1,70 +1,67 @@ -TakeOrderedAndProject -+- HashAggregate - +- CometNativeColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- Project - : : : : +- BroadcastHashJoin - : : : : :- Project - : : : : : +- BroadcastHashJoin - : : : : : :- Project - : : : : : : +- BroadcastHashJoin - : : : : : : :- Filter - : : : : : : : +- ColumnarToRow - : : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : : : +- SubqueryBroadcast - : : : : : : : +- BroadcastExchange - : : : : : : : +- CometNativeColumnarToRow - : : : : : : : +- CometProject - : : : : : : : +- CometFilter - : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : : : +- BroadcastExchange - : : : : : : +- Filter - : : : : : : +- ColumnarToRow - : : : : : : +- Scan parquet spark_catalog.default.store_returns [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometNativeColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : : +- BroadcastExchange - : : : : : +- Filter - : : : : : +- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : +- ReusedSubquery - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- CometNativeColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.store - +- BroadcastExchange - +- CometNativeColumnarToRow - +- CometProject - +- CometFilter - +- CometNativeScan parquet spark_catalog.default.item +CometNativeColumnarToRow ++- CometTakeOrderedAndProject + +- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometProject + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometProject + : : : : : : +- CometBroadcastHashJoin + : : : : : : :- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : : : : +- SubqueryBroadcast + : : : : : : : +- BroadcastExchange + : : : : : : : +- CometNativeColumnarToRow + : : : : : : : +- CometProject + : : : : : : : +- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : +- CometBroadcastExchange + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.store_returns + : : : : : : +- SubqueryBroadcast + : : : : : : +- BroadcastExchange + : : : : : : +- CometNativeColumnarToRow + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- CometBroadcastExchange + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometNativeColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.store + +- CometBroadcastExchange + +- CometProject + +- CometFilter + +- CometNativeScan parquet spark_catalog.default.item -Comet accelerated 22 out of 57 eligible operators (38%). Final plan contains 11 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 55 out of 61 eligible operators (90%). Final plan contains 4 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q26.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q26.native_datafusion/extended.txt index ceba8f5828..71fd3bbe6f 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q26.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q26.native_datafusion/extended.txt @@ -1,44 +1,39 @@ -TakeOrderedAndProject -+- HashAggregate - +- CometNativeColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- Filter - : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.customer_demographics - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- CometNativeColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.item - +- BroadcastExchange - +- CometNativeColumnarToRow - +- CometProject - +- CometFilter - +- CometNativeScan parquet spark_catalog.default.promotion +CometNativeColumnarToRow ++- CometTakeOrderedAndProject + +- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometNativeColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.customer_demographics + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.item + +- CometBroadcastExchange + +- CometProject + +- CometFilter + +- CometNativeScan parquet spark_catalog.default.promotion -Comet accelerated 16 out of 35 eligible operators (45%). Final plan contains 7 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 33 out of 35 eligible operators (94%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q27.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q27.native_datafusion/extended.txt index 20f7517d3e..15ab28f784 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q27.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q27.native_datafusion/extended.txt @@ -1,45 +1,40 @@ -TakeOrderedAndProject -+- HashAggregate - +- CometNativeColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Expand - +- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- Filter - : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.customer_demographics - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- CometNativeColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.store - +- BroadcastExchange - +- CometNativeColumnarToRow - +- CometProject - +- CometFilter - +- CometNativeScan parquet spark_catalog.default.item +CometNativeColumnarToRow ++- CometTakeOrderedAndProject + +- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometExpand + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometNativeColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.customer_demographics + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.store + +- CometBroadcastExchange + +- CometProject + +- CometFilter + +- CometNativeScan parquet spark_catalog.default.item -Comet accelerated 16 out of 36 eligible operators (44%). Final plan contains 7 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 34 out of 36 eligible operators (94%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q29.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q29.native_datafusion/extended.txt index bd9fa52aa7..f58c012650 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q29.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q29.native_datafusion/extended.txt @@ -1,75 +1,67 @@ -TakeOrderedAndProject -+- HashAggregate - +- CometNativeColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- Project - : : : : +- BroadcastHashJoin - : : : : :- Project - : : : : : +- BroadcastHashJoin - : : : : : :- Project - : : : : : : +- BroadcastHashJoin - : : : : : : :- Filter - : : : : : : : +- ColumnarToRow - : : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : : : +- SubqueryBroadcast - : : : : : : : +- BroadcastExchange - : : : : : : : +- CometNativeColumnarToRow - : : : : : : : +- CometProject - : : : : : : : +- CometFilter - : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : : : +- BroadcastExchange - : : : : : : +- Filter - : : : : : : +- ColumnarToRow - : : : : : : +- Scan parquet spark_catalog.default.store_returns [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometNativeColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : : +- BroadcastExchange - : : : : : +- Filter - : : : : : +- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- CometNativeColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.store - +- BroadcastExchange - +- CometNativeColumnarToRow - +- CometProject - +- CometFilter - +- CometNativeScan parquet spark_catalog.default.item +CometNativeColumnarToRow ++- CometTakeOrderedAndProject + +- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometProject + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometProject + : : : : : : +- CometBroadcastHashJoin + : : : : : : :- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : : : : +- SubqueryBroadcast + : : : : : : : +- BroadcastExchange + : : : : : : : +- CometNativeColumnarToRow + : : : : : : : +- CometProject + : : : : : : : +- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : +- CometBroadcastExchange + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.store_returns + : : : : : : +- SubqueryBroadcast + : : : : : : +- BroadcastExchange + : : : : : : +- CometNativeColumnarToRow + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- CometBroadcastExchange + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometNativeColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.store + +- CometBroadcastExchange + +- CometProject + +- CometFilter + +- CometNativeScan parquet spark_catalog.default.item -Comet accelerated 25 out of 61 eligible operators (40%). Final plan contains 12 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 55 out of 61 eligible operators (90%). Final plan contains 4 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q30.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q30.native_datafusion/extended.txt index c2e96eb412..18d0a12a03 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q30.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q30.native_datafusion/extended.txt @@ -1,75 +1,70 @@ -TakeOrderedAndProject -+- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Filter - : : : +- HashAggregate - : : : +- CometNativeColumnarToRow - : : : +- CometColumnarExchange - : : : +- HashAggregate - : : : +- Project - : : : +- BroadcastHashJoin - : : : :- Project - : : : : +- BroadcastHashJoin - : : : : :- Filter - : : : : : +- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.web_returns [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.customer_address - : : +- BroadcastExchange - : : +- Filter - : : +- HashAggregate - : : +- CometNativeColumnarToRow - : : +- CometColumnarExchange - : : +- HashAggregate - : : +- HashAggregate - : : +- CometNativeColumnarToRow - : : +- CometColumnarExchange - : : +- HashAggregate - : : +- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- Filter - : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.web_returns [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- ReusedSubquery - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.customer_address - : +- BroadcastExchange - : +- CometNativeColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.customer - +- BroadcastExchange - +- CometNativeColumnarToRow +CometNativeColumnarToRow ++- CometTakeOrderedAndProject + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometHashAggregate + : : : +- CometExchange + : : : +- CometHashAggregate + : : : +- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.web_returns + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometNativeColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.customer_address + : : +- CometBroadcastExchange + : : +- CometFilter + : : +- CometHashAggregate + : : +- CometExchange + : : +- CometHashAggregate + : : +- CometHashAggregate + : : +- CometExchange + : : +- CometHashAggregate + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.web_returns + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometNativeColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.customer_address + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.customer + +- CometBroadcastExchange +- CometProject +- CometFilter +- CometNativeScan parquet spark_catalog.default.customer_address -Comet accelerated 24 out of 61 eligible operators (39%). Final plan contains 12 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 61 out of 65 eligible operators (93%). Final plan contains 3 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q31.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q31.native_datafusion/extended.txt index 03bcf0413e..d747a5243e 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q31.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q31.native_datafusion/extended.txt @@ -1,150 +1,138 @@ CometNativeColumnarToRow +- CometSort - +- CometColumnarExchange - +- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- BroadcastHashJoin - : : : : :- HashAggregate - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometColumnarExchange - : : : : : +- HashAggregate - : : : : : +- Project - : : : : : +- BroadcastHashJoin - : : : : : :- Project - : : : : : : +- BroadcastHashJoin - : : : : : : :- Filter - : : : : : : : +- ColumnarToRow - : : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : : : +- SubqueryBroadcast - : : : : : : : +- BroadcastExchange - : : : : : : : +- CometNativeColumnarToRow - : : : : : : : +- CometFilter - : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : : : +- BroadcastExchange - : : : : : : +- CometNativeColumnarToRow - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : : +- BroadcastExchange - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.customer_address - : : : : +- BroadcastExchange - : : : : +- HashAggregate - : : : : +- CometNativeColumnarToRow - : : : : +- CometColumnarExchange - : : : : +- HashAggregate - : : : : +- Project - : : : : +- BroadcastHashJoin - : : : : :- Project - : : : : : +- BroadcastHashJoin - : : : : : :- Filter - : : : : : : +- ColumnarToRow - : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometNativeColumnarToRow - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : : +- BroadcastExchange - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.customer_address - : : : +- BroadcastExchange - : : : +- HashAggregate - : : : +- CometNativeColumnarToRow - : : : +- CometColumnarExchange - : : : +- HashAggregate - : : : +- Project - : : : +- BroadcastHashJoin - : : : :- Project - : : : : +- BroadcastHashJoin - : : : : :- Filter - : : : : : +- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.customer_address - : : +- BroadcastExchange - : : +- HashAggregate - : : +- CometNativeColumnarToRow - : : +- CometColumnarExchange - : : +- HashAggregate - : : +- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- Filter - : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- ReusedSubquery - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.customer_address - : +- BroadcastExchange - : +- HashAggregate - : +- CometNativeColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Filter - : : : +- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : +- ReusedSubquery - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- CometNativeColumnarToRow - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.customer_address - +- BroadcastExchange - +- HashAggregate - +- CometNativeColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- Filter - : : +- ColumnarToRow - : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : +- ReusedSubquery - : +- BroadcastExchange - : +- CometNativeColumnarToRow - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.date_dim - +- BroadcastExchange - +- CometNativeColumnarToRow - +- CometFilter - +- CometNativeScan parquet spark_catalog.default.customer_address + +- CometExchange + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometBroadcastHashJoin + : : : : :- CometHashAggregate + : : : : : +- CometExchange + : : : : : +- CometHashAggregate + : : : : : +- CometProject + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometProject + : : : : : : +- CometBroadcastHashJoin + : : : : : : :- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : : : : +- SubqueryBroadcast + : : : : : : : +- BroadcastExchange + : : : : : : : +- CometNativeColumnarToRow + : : : : : : : +- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : +- CometBroadcastExchange + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- CometBroadcastExchange + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.customer_address + : : : : +- CometBroadcastExchange + : : : : +- CometHashAggregate + : : : : +- CometExchange + : : : : +- CometHashAggregate + : : : : +- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometProject + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : : : +- SubqueryBroadcast + : : : : : : +- BroadcastExchange + : : : : : : +- CometNativeColumnarToRow + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- CometBroadcastExchange + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.customer_address + : : : +- CometBroadcastExchange + : : : +- CometHashAggregate + : : : +- CometExchange + : : : +- CometHashAggregate + : : : +- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometNativeColumnarToRow + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.customer_address + : : +- CometBroadcastExchange + : : +- CometHashAggregate + : : +- CometExchange + : : +- CometHashAggregate + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.web_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometNativeColumnarToRow + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.customer_address + : +- CometBroadcastExchange + : +- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.web_sales + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometNativeColumnarToRow + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.customer_address + +- CometBroadcastExchange + +- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.web_sales + : : +- SubqueryBroadcast + : : +- BroadcastExchange + : : +- CometNativeColumnarToRow + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.date_dim + +- CometBroadcastExchange + +- CometFilter + +- CometNativeScan parquet spark_catalog.default.customer_address -Comet accelerated 38 out of 120 eligible operators (31%). Final plan contains 28 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 117 out of 129 eligible operators (90%). Final plan contains 7 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q32.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q32.native_datafusion/extended.txt index f7096f5c3b..5d009dea2d 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q32.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q32.native_datafusion/extended.txt @@ -1,48 +1,47 @@ -HashAggregate -+- CometNativeColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Filter - : : : +- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.item - : +- BroadcastExchange - : +- Filter - : +- HashAggregate - : +- CometNativeColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- Filter - : : +- ColumnarToRow - : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : +- ReusedSubquery - : +- BroadcastExchange - : +- CometNativeColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.date_dim - +- BroadcastExchange - +- CometNativeColumnarToRow - +- CometProject - +- CometFilter - +- CometNativeScan parquet spark_catalog.default.date_dim +CometNativeColumnarToRow ++- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometNativeColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.item + : +- CometBroadcastExchange + : +- CometFilter + : +- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : +- SubqueryBroadcast + : : +- BroadcastExchange + : : +- CometNativeColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.date_dim + +- CometBroadcastExchange + +- CometProject + +- CometFilter + +- CometNativeScan parquet spark_catalog.default.date_dim -Comet accelerated 14 out of 38 eligible operators (36%). Final plan contains 8 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 38 out of 42 eligible operators (90%). Final plan contains 3 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q33.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q33.native_datafusion/extended.txt index 38b3efb123..6afaa68137 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q33.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q33.native_datafusion/extended.txt @@ -1,112 +1,107 @@ -TakeOrderedAndProject -+- HashAggregate - +- CometNativeColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Union - :- HashAggregate - : +- CometNativeColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- Filter - : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.customer_address - : +- BroadcastExchange - : +- CometNativeColumnarToRow - : +- CometBroadcastHashJoin - : :- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.item - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.item - :- HashAggregate - : +- CometNativeColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- Filter - : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- ReusedSubquery - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.customer_address - : +- BroadcastExchange - : +- CometNativeColumnarToRow - : +- CometBroadcastHashJoin - : :- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.item - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.item - +- HashAggregate - +- CometNativeColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Filter - : : : +- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : +- ReusedSubquery - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- CometNativeColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.customer_address - +- BroadcastExchange - +- CometNativeColumnarToRow - +- CometBroadcastHashJoin - :- CometFilter - : +- CometNativeScan parquet spark_catalog.default.item - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometNativeScan parquet spark_catalog.default.item +CometNativeColumnarToRow ++- CometTakeOrderedAndProject + +- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometUnion + :- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometNativeColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.customer_address + : +- CometBroadcastExchange + : +- CometBroadcastHashJoin + : :- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.item + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.item + :- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometNativeColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.customer_address + : +- CometBroadcastExchange + : +- CometBroadcastHashJoin + : :- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.item + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.item + +- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.web_sales + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometNativeColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.customer_address + +- CometBroadcastExchange + +- CometBroadcastHashJoin + :- CometFilter + : +- CometNativeScan parquet spark_catalog.default.item + +- CometBroadcastExchange + +- CometProject + +- CometFilter + +- CometNativeScan parquet spark_catalog.default.item -Comet accelerated 46 out of 93 eligible operators (49%). Final plan contains 17 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 95 out of 101 eligible operators (94%). Final plan contains 4 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q34.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q34.native_datafusion/extended.txt index 40230582bb..1b61be2937 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q34.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q34.native_datafusion/extended.txt @@ -1,47 +1,41 @@ CometNativeColumnarToRow +- CometSort - +- CometColumnarExchange - +- Project - +- BroadcastHashJoin - :- Filter - : +- HashAggregate - : +- CometNativeColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- Filter - : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.store - : +- BroadcastExchange - : +- CometNativeColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.household_demographics - +- BroadcastExchange - +- CometNativeColumnarToRow - +- CometProject - +- CometFilter - +- CometNativeScan parquet spark_catalog.default.customer + +- CometExchange + +- CometProject + +- CometBroadcastHashJoin + :- CometFilter + : +- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometNativeColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.store + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.household_demographics + +- CometBroadcastExchange + +- CometProject + +- CometFilter + +- CometNativeScan parquet spark_catalog.default.customer -Comet accelerated 18 out of 37 eligible operators (48%). Final plan contains 8 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 35 out of 37 eligible operators (94%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q35.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q35.native_datafusion/extended.txt index 40a8ef4a6e..2785c3d0b1 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q35.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q35.native_datafusion/extended.txt @@ -10,46 +10,52 @@ TakeOrderedAndProject : :- Project : : +- Filter : : +- BroadcastHashJoin - : : :- BroadcastHashJoin - : : : :- BroadcastHashJoin - : : : : :- CometNativeColumnarToRow - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.customer - : : : : +- BroadcastExchange - : : : : +- Project - : : : : +- BroadcastHashJoin - : : : : :- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow + : : :- BroadcastHashJoin [COMET: Unsupported join type ExistenceJoin(exists#1)] + : : : :- CometNativeColumnarToRow + : : : : +- CometBroadcastHashJoin + : : : : :- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.customer + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometNativeScan parquet spark_catalog.default.store_sales + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometNativeColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange : : : : +- CometProject : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- BroadcastExchange - : : : +- Project - : : : +- BroadcastHashJoin - : : : :- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- ReusedSubquery - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow + : : : +- CometNativeColumnarToRow + : : : +- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometNativeScan parquet spark_catalog.default.web_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometNativeColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- BroadcastExchange - : : +- Project - : : +- BroadcastHashJoin - : : :- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : +- ReusedSubquery - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow + : : +- CometNativeColumnarToRow + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometNativeColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange : : +- CometProject : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -64,4 +70,4 @@ TakeOrderedAndProject +- CometFilter +- CometNativeScan parquet spark_catalog.default.customer_demographics -Comet accelerated 21 out of 54 eligible operators (38%). Final plan contains 11 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 41 out of 62 eligible operators (66%). Final plan contains 9 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q36.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q36.native_datafusion/extended.txt index aa40ed68cc..05ce793d8d 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q36.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q36.native_datafusion/extended.txt @@ -3,41 +3,36 @@ TakeOrderedAndProject +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] +- CometNativeColumnarToRow +- CometSort - +- CometColumnarExchange - +- HashAggregate - +- CometNativeColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Expand - +- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Filter - : : : +- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- CometNativeColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.item - +- BroadcastExchange - +- CometNativeColumnarToRow - +- CometProject - +- CometFilter - +- CometNativeScan parquet spark_catalog.default.store + +- CometExchange + +- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometExpand + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometNativeColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.item + +- CometBroadcastExchange + +- CometProject + +- CometFilter + +- CometNativeScan parquet spark_catalog.default.store -Comet accelerated 15 out of 34 eligible operators (44%). Final plan contains 7 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 29 out of 34 eligible operators (85%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q37.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q37.native_datafusion/extended.txt index 296b2c8117..16e3a70d9b 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q37.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q37.native_datafusion/extended.txt @@ -1,38 +1,34 @@ CometNativeColumnarToRow +- CometTakeOrderedAndProject +- CometHashAggregate - +- CometColumnarExchange - +- HashAggregate - +- Project - +- BroadcastHashJoin - :- BroadcastExchange - : +- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.item - : : +- BroadcastExchange - : : +- Project - : : +- Filter - : : +- ColumnarToRow - : : +- Scan parquet spark_catalog.default.inventory [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- CometNativeColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.date_dim - +- CometNativeColumnarToRow - +- CometProject - +- CometFilter - +- CometNativeScan parquet spark_catalog.default.catalog_sales + +- CometExchange + +- CometHashAggregate + +- CometProject + +- CometBroadcastHashJoin + :- CometBroadcastExchange + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.item + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.inventory + : : +- SubqueryBroadcast + : : +- BroadcastExchange + : : +- CometNativeColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.date_dim + +- CometProject + +- CometFilter + +- CometNativeScan parquet spark_catalog.default.catalog_sales -Comet accelerated 15 out of 30 eligible operators (50%). Final plan contains 6 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 28 out of 30 eligible operators (93%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q38.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q38.native_datafusion/extended.txt index 6d9f5c7a17..29422273c5 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q38.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q38.native_datafusion/extended.txt @@ -6,74 +6,75 @@ CometNativeColumnarToRow +- CometBroadcastHashJoin :- CometBroadcastHashJoin : :- CometHashAggregate - : : +- CometColumnarExchange - : : +- HashAggregate - : : +- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- Filter - : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.customer + : : +- CometExchange + : : +- CometHashAggregate + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometNativeColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.customer : +- CometBroadcastExchange : +- CometHashAggregate - : +- CometColumnarExchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Filter - : : : +- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : +- ReusedSubquery - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- CometNativeColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.customer + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometNativeColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.customer +- CometBroadcastExchange +- CometHashAggregate - +- CometColumnarExchange - +- HashAggregate - +- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- Filter - : : +- ColumnarToRow - : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : +- ReusedSubquery - : +- BroadcastExchange - : +- CometNativeColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.date_dim - +- BroadcastExchange - +- CometNativeColumnarToRow - +- CometProject - +- CometFilter - +- CometNativeScan parquet spark_catalog.default.customer + +- CometExchange + +- CometHashAggregate + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.web_sales + : : +- SubqueryBroadcast + : : +- BroadcastExchange + : : +- CometNativeColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.date_dim + +- CometBroadcastExchange + +- CometProject + +- CometFilter + +- CometNativeScan parquet spark_catalog.default.customer -Comet accelerated 35 out of 66 eligible operators (53%). Final plan contains 11 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 68 out of 74 eligible operators (91%). Final plan contains 4 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q39a.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q39a.native_datafusion/extended.txt index d9cec17b28..eba1ff4dcd 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q39a.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q39a.native_datafusion/extended.txt @@ -1,75 +1,65 @@ CometNativeColumnarToRow +- CometSort - +- CometColumnarExchange - +- BroadcastHashJoin - :- Project - : +- Filter - : +- HashAggregate - : +- CometNativeColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- Filter - : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.inventory [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.item - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.warehouse - : +- BroadcastExchange - : +- CometNativeColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.date_dim - +- BroadcastExchange - +- Project - +- Filter - +- HashAggregate - +- CometNativeColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Filter - : : : +- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.inventory [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.item - : +- BroadcastExchange - : +- CometNativeColumnarToRow - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.warehouse - +- BroadcastExchange - +- CometNativeColumnarToRow - +- CometProject - +- CometFilter - +- CometNativeScan parquet spark_catalog.default.date_dim + +- CometExchange + +- CometBroadcastHashJoin + :- CometProject + : +- CometFilter + : +- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.inventory + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometNativeColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.item + : : +- CometBroadcastExchange + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.warehouse + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.date_dim + +- CometBroadcastExchange + +- CometProject + +- CometFilter + +- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.inventory + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometNativeColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.item + : +- CometBroadcastExchange + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.warehouse + +- CometBroadcastExchange + +- CometProject + +- CometFilter + +- CometNativeScan parquet spark_catalog.default.date_dim -Comet accelerated 24 out of 60 eligible operators (40%). Final plan contains 13 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 56 out of 60 eligible operators (93%). Final plan contains 3 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q39b.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q39b.native_datafusion/extended.txt index d9cec17b28..eba1ff4dcd 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q39b.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q39b.native_datafusion/extended.txt @@ -1,75 +1,65 @@ CometNativeColumnarToRow +- CometSort - +- CometColumnarExchange - +- BroadcastHashJoin - :- Project - : +- Filter - : +- HashAggregate - : +- CometNativeColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- Filter - : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.inventory [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.item - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.warehouse - : +- BroadcastExchange - : +- CometNativeColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.date_dim - +- BroadcastExchange - +- Project - +- Filter - +- HashAggregate - +- CometNativeColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Filter - : : : +- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.inventory [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.item - : +- BroadcastExchange - : +- CometNativeColumnarToRow - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.warehouse - +- BroadcastExchange - +- CometNativeColumnarToRow - +- CometProject - +- CometFilter - +- CometNativeScan parquet spark_catalog.default.date_dim + +- CometExchange + +- CometBroadcastHashJoin + :- CometProject + : +- CometFilter + : +- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.inventory + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometNativeColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.item + : : +- CometBroadcastExchange + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.warehouse + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.date_dim + +- CometBroadcastExchange + +- CometProject + +- CometFilter + +- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.inventory + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometNativeColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.item + : +- CometBroadcastExchange + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.warehouse + +- CometBroadcastExchange + +- CometProject + +- CometFilter + +- CometNativeScan parquet spark_catalog.default.date_dim -Comet accelerated 24 out of 60 eligible operators (40%). Final plan contains 13 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 56 out of 60 eligible operators (93%). Final plan contains 3 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q4.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q4.native_datafusion/extended.txt index 737217ff91..b2914584b4 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q4.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q4.native_datafusion/extended.txt @@ -1,154 +1,147 @@ -TakeOrderedAndProject -+- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- BroadcastHashJoin - : : : : :- Filter - : : : : : +- HashAggregate - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometColumnarExchange - : : : : : +- HashAggregate - : : : : : +- Project - : : : : : +- BroadcastHashJoin - : : : : : :- Project - : : : : : : +- BroadcastHashJoin - : : : : : : :- CometNativeColumnarToRow - : : : : : : : +- CometProject - : : : : : : : +- CometFilter - : : : : : : : +- CometNativeScan parquet spark_catalog.default.customer - : : : : : : +- BroadcastExchange - : : : : : : +- Filter - : : : : : : +- ColumnarToRow - : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometNativeColumnarToRow - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : : +- BroadcastExchange - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : +- BroadcastExchange - : : : : +- HashAggregate - : : : : +- CometNativeColumnarToRow - : : : : +- CometColumnarExchange - : : : : +- HashAggregate - : : : : +- Project - : : : : +- BroadcastHashJoin - : : : : :- Project - : : : : : +- BroadcastHashJoin - : : : : : :- CometNativeColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.customer - : : : : : +- BroadcastExchange - : : : : : +- Filter - : : : : : +- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- Filter - : : : +- HashAggregate - : : : +- CometNativeColumnarToRow - : : : +- CometColumnarExchange - : : : +- HashAggregate - : : : +- Project - : : : +- BroadcastHashJoin - : : : :- Project - : : : : +- BroadcastHashJoin - : : : : :- CometNativeColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.customer - : : : : +- BroadcastExchange - : : : : +- Filter - : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- ReusedSubquery - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- HashAggregate - : : +- CometNativeColumnarToRow - : : +- CometColumnarExchange - : : +- HashAggregate - : : +- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.customer - : : : +- BroadcastExchange - : : : +- Filter - : : : +- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : +- ReusedSubquery - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- Filter - : +- HashAggregate - : +- CometNativeColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.customer - : : +- BroadcastExchange - : : +- Filter - : : +- ColumnarToRow - : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : +- ReusedSubquery - : +- BroadcastExchange - : +- CometNativeColumnarToRow - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.date_dim - +- BroadcastExchange - +- HashAggregate - +- CometNativeColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- CometNativeColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.customer - : +- BroadcastExchange - : +- Filter - : +- ColumnarToRow - : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : +- ReusedSubquery - +- BroadcastExchange - +- CometNativeColumnarToRow - +- CometFilter - +- CometNativeScan parquet spark_catalog.default.date_dim +CometNativeColumnarToRow ++- CometTakeOrderedAndProject + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometBroadcastHashJoin + : : : : :- CometFilter + : : : : : +- CometHashAggregate + : : : : : +- CometExchange + : : : : : +- CometHashAggregate + : : : : : +- CometProject + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometProject + : : : : : : +- CometBroadcastHashJoin + : : : : : : :- CometProject + : : : : : : : +- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.customer + : : : : : : +- CometBroadcastExchange + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : : : +- SubqueryBroadcast + : : : : : : +- BroadcastExchange + : : : : : : +- CometNativeColumnarToRow + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- CometBroadcastExchange + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometHashAggregate + : : : : +- CometExchange + : : : : +- CometHashAggregate + : : : : +- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometProject + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.customer + : : : : : +- CometBroadcastExchange + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometNativeColumnarToRow + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometFilter + : : : +- CometHashAggregate + : : : +- CometExchange + : : : +- CometHashAggregate + : : : +- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.customer + : : : : +- CometBroadcastExchange + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometNativeColumnarToRow + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometHashAggregate + : : +- CometExchange + : : +- CometHashAggregate + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.customer + : : : +- CometBroadcastExchange + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometNativeColumnarToRow + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometFilter + : +- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.customer + : : +- CometBroadcastExchange + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.web_sales + : : +- SubqueryBroadcast + : : +- BroadcastExchange + : : +- CometNativeColumnarToRow + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.date_dim + +- CometBroadcastExchange + +- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.customer + : +- CometBroadcastExchange + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.web_sales + : +- SubqueryBroadcast + : +- BroadcastExchange + : +- CometNativeColumnarToRow + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.date_dim + +- CometBroadcastExchange + +- CometFilter + +- CometNativeScan parquet spark_catalog.default.date_dim -Comet accelerated 40 out of 126 eligible operators (31%). Final plan contains 26 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 126 out of 138 eligible operators (91%). Final plan contains 7 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q40.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q40.native_datafusion/extended.txt index f1d98d3059..9b00a2c6a4 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q40.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q40.native_datafusion/extended.txt @@ -12,15 +12,14 @@ CometNativeColumnarToRow : : :- CometProject : : : +- CometSortMergeJoin : : : :- CometSort - : : : : +- CometColumnarExchange - : : : : +- Filter - : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometExchange + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometNativeColumnarToRow + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- CometSort : : : +- CometExchange : : : +- CometProject @@ -38,4 +37,4 @@ CometNativeColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.date_dim -Comet accelerated 32 out of 36 eligible operators (88%). Final plan contains 3 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 34 out of 36 eligible operators (94%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q45.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q45.native_datafusion/extended.txt index 5b686ba177..7292a238d5 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q45.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q45.native_datafusion/extended.txt @@ -5,40 +5,36 @@ TakeOrderedAndProject +- HashAggregate +- Project +- Filter - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- Project - : : : : +- BroadcastHashJoin - : : : : :- Filter - : : : : : +- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.customer - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.customer_address - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- CometNativeColumnarToRow + +- BroadcastHashJoin [COMET: Unsupported join type ExistenceJoin(exists#1)] + :- CometNativeColumnarToRow + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.web_sales + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometNativeColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.customer + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.customer_address + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange : +- CometProject : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.item @@ -48,4 +44,4 @@ TakeOrderedAndProject +- CometFilter +- CometNativeScan parquet spark_catalog.default.item -Comet accelerated 18 out of 41 eligible operators (43%). Final plan contains 8 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 32 out of 41 eligible operators (78%). Final plan contains 4 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q46.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q46.native_datafusion/extended.txt index cebcd85144..dbdaaa275a 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q46.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q46.native_datafusion/extended.txt @@ -1,56 +1,49 @@ -TakeOrderedAndProject -+- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- HashAggregate - : : +- CometNativeColumnarToRow - : : +- CometColumnarExchange - : : +- HashAggregate - : : +- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- Project - : : : : +- BroadcastHashJoin - : : : : :- Project - : : : : : +- BroadcastHashJoin - : : : : : :- Filter - : : : : : : +- ColumnarToRow - : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometNativeColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : : +- BroadcastExchange - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.store - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.household_demographics - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.customer_address - : +- BroadcastExchange - : +- CometNativeColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.customer - +- BroadcastExchange - +- CometNativeColumnarToRow +CometNativeColumnarToRow ++- CometTakeOrderedAndProject + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometHashAggregate + : : +- CometExchange + : : +- CometHashAggregate + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometProject + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : : : +- SubqueryBroadcast + : : : : : : +- BroadcastExchange + : : : : : : +- CometNativeColumnarToRow + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- CometBroadcastExchange + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.store + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.household_demographics + : : +- CometBroadcastExchange + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.customer_address + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.customer + +- CometBroadcastExchange +- CometFilter +- CometNativeScan parquet spark_catalog.default.customer_address -Comet accelerated 20 out of 45 eligible operators (44%). Final plan contains 9 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 43 out of 45 eligible operators (95%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q47.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q47.native_datafusion/extended.txt index ab44bbef5c..0886059452 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q47.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q47.native_datafusion/extended.txt @@ -10,111 +10,96 @@ TakeOrderedAndProject : : +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] : : +- CometNativeColumnarToRow : : +- CometSort - : : +- CometColumnarExchange - : : +- HashAggregate - : : +- CometNativeColumnarToRow - : : +- CometColumnarExchange - : : +- HashAggregate - : : +- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- Project - : : : : +- BroadcastHashJoin - : : : : :- CometNativeColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : +- BroadcastExchange - : : : : +- Filter - : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.store + : : +- CometExchange + : : +- CometHashAggregate + : : +- CometExchange + : : +- CometHashAggregate + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : : +- CometBroadcastExchange + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometNativeColumnarToRow + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.store : +- BroadcastExchange : +- Project : +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] : +- CometNativeColumnarToRow : +- CometSort - : +- CometColumnarExchange - : +- HashAggregate - : +- CometNativeColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : +- BroadcastExchange - : : : +- Filter - : : : +- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- CometNativeColumnarToRow - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.store + : +- CometExchange + : +- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : +- CometBroadcastExchange + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometNativeColumnarToRow + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.store +- BroadcastExchange +- Project +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] +- CometNativeColumnarToRow +- CometSort - +- CometColumnarExchange - +- HashAggregate - +- CometNativeColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.item - : : +- BroadcastExchange - : : +- Filter - : : +- ColumnarToRow - : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- CometNativeColumnarToRow - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.date_dim - +- BroadcastExchange - +- CometNativeColumnarToRow - +- CometFilter - +- CometNativeScan parquet spark_catalog.default.store + +- CometExchange + +- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.item + : : +- CometBroadcastExchange + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : +- SubqueryBroadcast + : : +- BroadcastExchange + : : +- CometNativeColumnarToRow + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.date_dim + +- CometBroadcastExchange + +- CometFilter + +- CometNativeScan parquet spark_catalog.default.store -Comet accelerated 36 out of 97 eligible operators (37%). Final plan contains 21 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 75 out of 97 eligible operators (77%). Final plan contains 6 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q48.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q48.native_datafusion/extended.txt index 5e6a26863a..75a1c725c2 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q48.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q48.native_datafusion/extended.txt @@ -1,42 +1,37 @@ -HashAggregate -+- CometNativeColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- Filter - : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.store - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.customer_demographics - : +- BroadcastExchange - : +- CometNativeColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.customer_address - +- BroadcastExchange - +- CometNativeColumnarToRow - +- CometProject - +- CometFilter - +- CometNativeScan parquet spark_catalog.default.date_dim +CometNativeColumnarToRow ++- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometNativeColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.store + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.customer_demographics + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.customer_address + +- CometBroadcastExchange + +- CometProject + +- CometFilter + +- CometNativeScan parquet spark_catalog.default.date_dim -Comet accelerated 15 out of 33 eligible operators (45%). Final plan contains 7 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 31 out of 33 eligible operators (93%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q49.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q49.native_datafusion/extended.txt index 0090739d55..6d7dadcde9 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q49.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q49.native_datafusion/extended.txt @@ -11,35 +11,31 @@ CometNativeColumnarToRow : +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] : +- CometNativeColumnarToRow : +- CometSort - : +- CometColumnarExchange - : +- HashAggregate - : +- CometNativeColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- BroadcastExchange - : : : +- Project - : : : +- Filter - : : : +- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- CometNativeColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.web_returns - : +- BroadcastExchange - : +- CometNativeColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometExchange + : +- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.web_sales + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometNativeColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.web_returns + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.date_dim :- Project : +- Filter : +- Window @@ -47,30 +43,31 @@ CometNativeColumnarToRow : +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] : +- CometNativeColumnarToRow : +- CometSort - : +- CometColumnarExchange - : +- HashAggregate - : +- CometNativeColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- BroadcastExchange - : : : +- Project - : : : +- Filter - : : : +- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : +- ReusedSubquery - : : +- CometNativeColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.catalog_returns - : +- BroadcastExchange - : +- CometNativeColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometExchange + : +- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometNativeColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.catalog_returns + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.date_dim +- Project +- Filter +- Window @@ -78,29 +75,30 @@ CometNativeColumnarToRow +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] +- CometNativeColumnarToRow +- CometSort - +- CometColumnarExchange - +- HashAggregate - +- CometNativeColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- BroadcastExchange - : : +- Project - : : +- Filter - : : +- ColumnarToRow - : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : +- ReusedSubquery - : +- CometNativeColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.store_returns - +- BroadcastExchange - +- CometNativeColumnarToRow - +- CometProject - +- CometFilter - +- CometNativeScan parquet spark_catalog.default.date_dim + +- CometExchange + +- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : +- SubqueryBroadcast + : : +- BroadcastExchange + : : +- CometNativeColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.store_returns + +- CometBroadcastExchange + +- CometProject + +- CometFilter + +- CometNativeScan parquet spark_catalog.default.date_dim -Comet accelerated 33 out of 87 eligible operators (37%). Final plan contains 17 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 72 out of 95 eligible operators (75%). Final plan contains 7 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q5.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q5.native_datafusion/extended.txt index 15ad014521..719229a9a3 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q5.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q5.native_datafusion/extended.txt @@ -1,106 +1,115 @@ -TakeOrderedAndProject -+- HashAggregate - +- CometNativeColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Expand - +- Union - :- HashAggregate - : +- CometNativeColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Union - : : : :- Project - : : : : +- Filter - : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- Project - : : : +- Filter - : : : +- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.store_returns [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : +- ReusedSubquery - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- CometNativeColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.store - :- HashAggregate - : +- CometNativeColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Union - : : : :- Project - : : : : +- Filter - : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- ReusedSubquery - : : : +- Project - : : : +- Filter - : : : +- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.catalog_returns [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : +- ReusedSubquery - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- CometNativeColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.catalog_page - +- HashAggregate - +- CometNativeColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- Union - : : :- Project - : : : +- Filter - : : : +- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : +- ReusedSubquery - : : +- Project - : : +- BroadcastHashJoin - : : :- BroadcastExchange - : : : +- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.web_returns [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : +- ReusedSubquery - : : +- CometNativeColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.web_sales - : +- BroadcastExchange - : +- CometNativeColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.date_dim - +- BroadcastExchange - +- CometNativeColumnarToRow - +- CometProject - +- CometFilter - +- CometNativeScan parquet spark_catalog.default.web_site +CometNativeColumnarToRow ++- CometTakeOrderedAndProject + +- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometExpand + +- CometUnion + :- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometUnion + : : : :- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometNativeColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.store_returns + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometNativeColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.store + :- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometUnion + : : : :- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometNativeColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.catalog_returns + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometNativeColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.catalog_page + +- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometUnion + : : :- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.web_sales + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometNativeColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometBroadcastExchange + : : : +- CometNativeScan parquet spark_catalog.default.web_returns + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometNativeColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.web_sales + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.date_dim + +- CometBroadcastExchange + +- CometProject + +- CometFilter + +- CometNativeScan parquet spark_catalog.default.web_site -Comet accelerated 28 out of 86 eligible operators (32%). Final plan contains 18 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 94 out of 106 eligible operators (88%). Final plan contains 7 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q50.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q50.native_datafusion/extended.txt index ab41730455..c013381de0 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q50.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q50.native_datafusion/extended.txt @@ -1,42 +1,37 @@ -TakeOrderedAndProject -+- HashAggregate - +- CometNativeColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- CometNativeColumnarToRow - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : +- BroadcastExchange - : : : +- Filter - : : : +- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.store_returns [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.store - : +- BroadcastExchange - : +- CometNativeColumnarToRow - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.date_dim - +- BroadcastExchange - +- CometNativeColumnarToRow - +- CometProject - +- CometFilter - +- CometNativeScan parquet spark_catalog.default.date_dim +CometNativeColumnarToRow ++- CometTakeOrderedAndProject + +- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : +- CometBroadcastExchange + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.store_returns + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometNativeColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.store + : +- CometBroadcastExchange + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.date_dim + +- CometBroadcastExchange + +- CometProject + +- CometFilter + +- CometNativeScan parquet spark_catalog.default.date_dim -Comet accelerated 14 out of 33 eligible operators (42%). Final plan contains 7 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 31 out of 33 eligible operators (93%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q51.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q51.native_datafusion/extended.txt index db4e38a1bb..425e772118 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q51.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q51.native_datafusion/extended.txt @@ -12,48 +12,47 @@ TakeOrderedAndProject : +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] : +- CometNativeColumnarToRow : +- CometSort - : +- CometColumnarExchange - : +- HashAggregate - : +- CometNativeColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- Filter - : : +- ColumnarToRow - : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- CometNativeColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometExchange + : +- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.web_sales + : : +- SubqueryBroadcast + : : +- BroadcastExchange + : : +- CometNativeColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.date_dim +- CometSort +- CometColumnarExchange +- Project +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] +- CometNativeColumnarToRow +- CometSort - +- CometColumnarExchange - +- HashAggregate - +- CometNativeColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Project - +- BroadcastHashJoin - :- Filter - : +- ColumnarToRow - : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : +- ReusedSubquery - +- BroadcastExchange - +- CometNativeColumnarToRow - +- CometProject - +- CometFilter - +- CometNativeScan parquet spark_catalog.default.date_dim + +- CometExchange + +- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometProject + +- CometBroadcastHashJoin + :- CometFilter + : +- CometNativeScan parquet spark_catalog.default.store_sales + : +- SubqueryBroadcast + : +- BroadcastExchange + : +- CometNativeColumnarToRow + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.date_dim + +- CometBroadcastExchange + +- CometProject + +- CometFilter + +- CometNativeScan parquet spark_catalog.default.date_dim -Comet accelerated 23 out of 47 eligible operators (48%). Final plan contains 10 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 40 out of 51 eligible operators (78%). Final plan contains 5 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q53.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q53.native_datafusion/extended.txt index 5c7a26057d..936cde052f 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q53.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q53.native_datafusion/extended.txt @@ -4,39 +4,34 @@ TakeOrderedAndProject +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] +- CometNativeColumnarToRow +- CometSort - +- CometColumnarExchange - +- HashAggregate - +- CometNativeColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.item - : : +- BroadcastExchange - : : +- Filter - : : +- ColumnarToRow - : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- CometNativeColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.date_dim - +- BroadcastExchange - +- CometNativeColumnarToRow - +- CometFilter - +- CometNativeScan parquet spark_catalog.default.store + +- CometExchange + +- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.item + : : +- CometBroadcastExchange + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : +- SubqueryBroadcast + : : +- BroadcastExchange + : : +- CometNativeColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.date_dim + +- CometBroadcastExchange + +- CometFilter + +- CometNativeScan parquet spark_catalog.default.store -Comet accelerated 14 out of 33 eligible operators (42%). Final plan contains 7 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 27 out of 33 eligible operators (81%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q54.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q54.native_datafusion/extended.txt index 273efea475..e7fd40210f 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q54.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q54.native_datafusion/extended.txt @@ -1,116 +1,96 @@ -TakeOrderedAndProject -+- HashAggregate - +- CometNativeColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- HashAggregate - +- CometNativeColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- CometNativeColumnarToRow - : : : : +- CometHashAggregate - : : : : +- CometColumnarExchange - : : : : +- HashAggregate - : : : : +- Project - : : : : +- BroadcastHashJoin - : : : : :- Project - : : : : : +- BroadcastHashJoin - : : : : : :- Project - : : : : : : +- BroadcastHashJoin - : : : : : : :- Union - : : : : : : : :- Project - : : : : : : : : +- Filter - : : : : : : : : +- ColumnarToRow - : : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : : : : +- SubqueryBroadcast - : : : : : : : : +- BroadcastExchange - : : : : : : : : +- CometNativeColumnarToRow - : : : : : : : : +- CometProject - : : : : : : : : +- CometFilter - : : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : : : : +- Project - : : : : : : : +- Filter - : : : : : : : +- ColumnarToRow - : : : : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : : : +- ReusedSubquery - : : : : : : +- BroadcastExchange - : : : : : : +- CometNativeColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : : +- BroadcastExchange - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.customer - : : : +- BroadcastExchange - : : : +- Filter - : : : +- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : : :- Subquery - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometHashAggregate - : : : : : +- CometExchange - : : : : : +- CometHashAggregate - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : +- Subquery - : : : : +- CometNativeColumnarToRow - : : : : +- CometHashAggregate - : : : : +- CometExchange - : : : : +- CometHashAggregate - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.customer_address - : +- BroadcastExchange - : +- CometNativeColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.store - +- BroadcastExchange - +- CometNativeColumnarToRow - +- CometProject - +- CometFilter - : :- Subquery - : : +- CometNativeColumnarToRow - : : +- CometHashAggregate - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- Subquery - : +- CometNativeColumnarToRow - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.date_dim - +- CometNativeScan parquet spark_catalog.default.date_dim +CometNativeColumnarToRow ++- CometTakeOrderedAndProject + +- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometHashAggregate + : : : : +- CometExchange + : : : : +- CometHashAggregate + : : : : +- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometProject + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometProject + : : : : : : +- CometBroadcastHashJoin + : : : : : : :- CometUnion + : : : : : : : :- CometProject + : : : : : : : : +- CometFilter + : : : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : : : : : +- SubqueryBroadcast + : : : : : : : : +- BroadcastExchange + : : : : : : : : +- CometNativeColumnarToRow + : : : : : : : : +- CometProject + : : : : : : : : +- CometFilter + : : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : : +- CometProject + : : : : : : : +- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.web_sales + : : : : : : : +- SubqueryBroadcast + : : : : : : : +- BroadcastExchange + : : : : : : : +- CometNativeColumnarToRow + : : : : : : : +- CometProject + : : : : : : : +- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : +- CometBroadcastExchange + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : : : +- CometBroadcastExchange + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.customer + : : : +- CometBroadcastExchange + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometNativeColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : : :- Subquery + : : : : : +- CometNativeColumnarToRow + : : : : : +- CometHashAggregate + : : : : : +- CometExchange + : : : : : +- CometHashAggregate + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- Subquery + : : : : +- CometNativeColumnarToRow + : : : : +- CometHashAggregate + : : : : +- CometExchange + : : : : +- CometHashAggregate + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.customer_address + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.store + +- CometBroadcastExchange + +- CometProject + +- CometFilter + : :- ReusedSubquery + : +- ReusedSubquery + +- CometNativeScan parquet spark_catalog.default.date_dim -Comet accelerated 51 out of 96 eligible operators (53%). Final plan contains 18 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 78 out of 88 eligible operators (88%). Final plan contains 6 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q56.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q56.native_datafusion/extended.txt index ad055b48b6..d52c40d57b 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q56.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q56.native_datafusion/extended.txt @@ -1,115 +1,110 @@ -TakeOrderedAndProject -+- HashAggregate - +- CometNativeColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Union - :- HashAggregate - : +- CometNativeColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- Filter - : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.customer_address - : +- BroadcastExchange - : +- CometNativeColumnarToRow - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.item - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.item - :- HashAggregate - : +- CometNativeColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- Filter - : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- ReusedSubquery - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.customer_address - : +- BroadcastExchange - : +- CometNativeColumnarToRow - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.item - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.item - +- HashAggregate - +- CometNativeColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Filter - : : : +- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : +- ReusedSubquery - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- CometNativeColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.customer_address - +- BroadcastExchange - +- CometNativeColumnarToRow - +- CometProject - +- CometBroadcastHashJoin - :- CometFilter - : +- CometNativeScan parquet spark_catalog.default.item - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometNativeScan parquet spark_catalog.default.item +CometNativeColumnarToRow ++- CometTakeOrderedAndProject + +- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometUnion + :- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometNativeColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.customer_address + : +- CometBroadcastExchange + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.item + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.item + :- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometNativeColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.customer_address + : +- CometBroadcastExchange + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.item + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.item + +- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.web_sales + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometNativeColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.customer_address + +- CometBroadcastExchange + +- CometProject + +- CometBroadcastHashJoin + :- CometFilter + : +- CometNativeScan parquet spark_catalog.default.item + +- CometBroadcastExchange + +- CometProject + +- CometFilter + +- CometNativeScan parquet spark_catalog.default.item -Comet accelerated 49 out of 96 eligible operators (51%). Final plan contains 17 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 98 out of 104 eligible operators (94%). Final plan contains 4 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q57.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q57.native_datafusion/extended.txt index 66c510aaa0..024bfeeeea 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q57.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q57.native_datafusion/extended.txt @@ -10,111 +10,96 @@ TakeOrderedAndProject : : +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] : : +- CometNativeColumnarToRow : : +- CometSort - : : +- CometColumnarExchange - : : +- HashAggregate - : : +- CometNativeColumnarToRow - : : +- CometColumnarExchange - : : +- HashAggregate - : : +- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- Project - : : : : +- BroadcastHashJoin - : : : : :- CometNativeColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : +- BroadcastExchange - : : : : +- Filter - : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.call_center + : : +- CometExchange + : : +- CometHashAggregate + : : +- CometExchange + : : +- CometHashAggregate + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : : +- CometBroadcastExchange + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometNativeColumnarToRow + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.call_center : +- BroadcastExchange : +- Project : +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] : +- CometNativeColumnarToRow : +- CometSort - : +- CometColumnarExchange - : +- HashAggregate - : +- CometNativeColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : +- BroadcastExchange - : : : +- Filter - : : : +- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- CometNativeColumnarToRow - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.call_center + : +- CometExchange + : +- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : +- CometBroadcastExchange + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometNativeColumnarToRow + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.call_center +- BroadcastExchange +- Project +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] +- CometNativeColumnarToRow +- CometSort - +- CometColumnarExchange - +- HashAggregate - +- CometNativeColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.item - : : +- BroadcastExchange - : : +- Filter - : : +- ColumnarToRow - : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- CometNativeColumnarToRow - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.date_dim - +- BroadcastExchange - +- CometNativeColumnarToRow - +- CometFilter - +- CometNativeScan parquet spark_catalog.default.call_center + +- CometExchange + +- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.item + : : +- CometBroadcastExchange + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : +- SubqueryBroadcast + : : +- BroadcastExchange + : : +- CometNativeColumnarToRow + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.date_dim + +- CometBroadcastExchange + +- CometFilter + +- CometNativeScan parquet spark_catalog.default.call_center -Comet accelerated 36 out of 97 eligible operators (37%). Final plan contains 21 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 75 out of 97 eligible operators (77%). Final plan contains 6 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q58.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q58.native_datafusion/extended.txt index be602e0f0a..6a2729349e 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q58.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q58.native_datafusion/extended.txt @@ -1,126 +1,134 @@ -TakeOrderedAndProject -+- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- Filter - : : +- HashAggregate - : : +- CometNativeColumnarToRow - : : +- CometColumnarExchange - : : +- HashAggregate - : : +- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- Filter - : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : : +- Subquery - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.item - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : : +- Subquery - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- Filter - : +- HashAggregate - : +- CometNativeColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Filter - : : : +- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : : +- ReusedSubquery - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.item - : +- BroadcastExchange - : +- CometNativeColumnarToRow - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : : +- ReusedSubquery - : +- CometNativeScan parquet spark_catalog.default.date_dim - +- BroadcastExchange - +- Filter - +- HashAggregate - +- CometNativeColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- Filter - : : +- ColumnarToRow - : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : +- ReusedSubquery - : +- BroadcastExchange - : +- CometNativeColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.item - +- BroadcastExchange - +- CometNativeColumnarToRow - +- CometProject - +- CometBroadcastHashJoin - :- CometFilter - : +- CometNativeScan parquet spark_catalog.default.date_dim - +- CometBroadcastExchange - +- CometProject - +- CometFilter - : +- ReusedSubquery - +- CometNativeScan parquet spark_catalog.default.date_dim +CometNativeColumnarToRow ++- CometTakeOrderedAndProject + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometFilter + : : +- CometHashAggregate + : : +- CometExchange + : : +- CometHashAggregate + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometNativeColumnarToRow + : : : : +- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : : +- Subquery + : : : : : +- CometNativeColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.item + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : : +- Subquery + : : : +- CometNativeColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometFilter + : +- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometNativeColumnarToRow + : : : +- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : : +- Subquery + : : : : +- CometNativeColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.item + : +- CometBroadcastExchange + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : : +- ReusedSubquery + : +- CometNativeScan parquet spark_catalog.default.date_dim + +- CometBroadcastExchange + +- CometFilter + +- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.web_sales + : : +- SubqueryBroadcast + : : +- BroadcastExchange + : : +- CometNativeColumnarToRow + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : : +- Subquery + : : : +- CometNativeColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.item + +- CometBroadcastExchange + +- CometProject + +- CometBroadcastHashJoin + :- CometFilter + : +- CometNativeScan parquet spark_catalog.default.date_dim + +- CometBroadcastExchange + +- CometProject + +- CometFilter + : +- ReusedSubquery + +- CometNativeScan parquet spark_catalog.default.date_dim -Comet accelerated 58 out of 108 eligible operators (53%). Final plan contains 16 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 112 out of 124 eligible operators (90%). Final plan contains 8 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q6.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q6.native_datafusion/extended.txt index f2cdb50e03..1cb21cb650 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q6.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q6.native_datafusion/extended.txt @@ -1,68 +1,57 @@ -TakeOrderedAndProject -+- Filter - +- HashAggregate - +- CometNativeColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.customer_address - : : : +- CometBroadcastExchange - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.customer - : : +- BroadcastExchange - : : +- Filter - : : +- ColumnarToRow - : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometProject - : : +- CometFilter - : : : +- Subquery - : : : +- CometNativeColumnarToRow - : : : +- CometHashAggregate - : : : +- CometExchange - : : : +- CometHashAggregate - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- CometNativeColumnarToRow - : +- CometProject - : +- CometFilter - : : +- Subquery - : : +- CometNativeColumnarToRow - : : +- CometHashAggregate - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- CometNativeScan parquet spark_catalog.default.date_dim - +- BroadcastExchange - +- CometNativeColumnarToRow - +- CometProject - +- CometBroadcastHashJoin - :- CometFilter - : +- CometNativeScan parquet spark_catalog.default.item - +- CometBroadcastExchange - +- CometFilter - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometFilter - +- CometNativeScan parquet spark_catalog.default.item +CometNativeColumnarToRow ++- CometTakeOrderedAndProject + +- CometFilter + +- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.customer_address + : : : +- CometBroadcastExchange + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.customer + : : +- CometBroadcastExchange + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : +- SubqueryBroadcast + : : +- BroadcastExchange + : : +- CometNativeColumnarToRow + : : +- CometProject + : : +- CometFilter + : : : +- Subquery + : : : +- CometNativeColumnarToRow + : : : +- CometHashAggregate + : : : +- CometExchange + : : : +- CometHashAggregate + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : : +- ReusedSubquery + : +- CometNativeScan parquet spark_catalog.default.date_dim + +- CometBroadcastExchange + +- CometProject + +- CometBroadcastHashJoin + :- CometFilter + : +- CometNativeScan parquet spark_catalog.default.item + +- CometBroadcastExchange + +- CometFilter + +- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometProject + +- CometFilter + +- CometNativeScan parquet spark_catalog.default.item -Comet accelerated 39 out of 58 eligible operators (67%). Final plan contains 8 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 48 out of 52 eligible operators (92%). Final plan contains 3 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q60.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q60.native_datafusion/extended.txt index ad055b48b6..d52c40d57b 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q60.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q60.native_datafusion/extended.txt @@ -1,115 +1,110 @@ -TakeOrderedAndProject -+- HashAggregate - +- CometNativeColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Union - :- HashAggregate - : +- CometNativeColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- Filter - : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.customer_address - : +- BroadcastExchange - : +- CometNativeColumnarToRow - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.item - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.item - :- HashAggregate - : +- CometNativeColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- Filter - : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- ReusedSubquery - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.customer_address - : +- BroadcastExchange - : +- CometNativeColumnarToRow - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.item - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.item - +- HashAggregate - +- CometNativeColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Filter - : : : +- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : +- ReusedSubquery - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- CometNativeColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.customer_address - +- BroadcastExchange - +- CometNativeColumnarToRow - +- CometProject - +- CometBroadcastHashJoin - :- CometFilter - : +- CometNativeScan parquet spark_catalog.default.item - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometNativeScan parquet spark_catalog.default.item +CometNativeColumnarToRow ++- CometTakeOrderedAndProject + +- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometUnion + :- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometNativeColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.customer_address + : +- CometBroadcastExchange + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.item + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.item + :- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometNativeColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.customer_address + : +- CometBroadcastExchange + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.item + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.item + +- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.web_sales + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometNativeColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.customer_address + +- CometBroadcastExchange + +- CometProject + +- CometBroadcastHashJoin + :- CometFilter + : +- CometNativeScan parquet spark_catalog.default.item + +- CometBroadcastExchange + +- CometProject + +- CometFilter + +- CometNativeScan parquet spark_catalog.default.item -Comet accelerated 49 out of 96 eligible operators (51%). Final plan contains 17 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 98 out of 104 eligible operators (94%). Final plan contains 4 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q61.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q61.native_datafusion/extended.txt index 025280d4ca..ef8907c7d8 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q61.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q61.native_datafusion/extended.txt @@ -1,101 +1,93 @@ Project -+- BroadcastNestedLoopJoin - :- HashAggregate - : +- CometNativeColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- Project - : : : : +- BroadcastHashJoin - : : : : :- Project - : : : : : +- BroadcastHashJoin - : : : : : :- Project - : : : : : : +- BroadcastHashJoin - : : : : : : :- Filter - : : : : : : : +- ColumnarToRow - : : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : : : +- SubqueryBroadcast - : : : : : : : +- BroadcastExchange - : : : : : : : +- CometNativeColumnarToRow - : : : : : : : +- CometProject - : : : : : : : +- CometFilter - : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : : : +- BroadcastExchange - : : : : : : +- CometNativeColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.store - : : : : : +- BroadcastExchange - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.promotion - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.customer - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.customer_address - : +- BroadcastExchange - : +- CometNativeColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.item ++- BroadcastNestedLoopJoin [COMET: BroadcastNestedLoopJoin is not supported] + :- CometNativeColumnarToRow + : +- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometProject + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometProject + : : : : : : +- CometBroadcastHashJoin + : : : : : : :- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : : : : +- SubqueryBroadcast + : : : : : : : +- BroadcastExchange + : : : : : : : +- CometNativeColumnarToRow + : : : : : : : +- CometProject + : : : : : : : +- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : +- CometBroadcastExchange + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.store + : : : : : +- CometBroadcastExchange + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.promotion + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.customer + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.customer_address + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.item +- BroadcastExchange - +- HashAggregate - +- CometNativeColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- Project - : : : : +- BroadcastHashJoin - : : : : :- Filter - : : : : : +- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : +- ReusedSubquery - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.store - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.customer - : +- BroadcastExchange - : +- CometNativeColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.customer_address - +- BroadcastExchange - +- CometNativeColumnarToRow - +- CometProject - +- CometFilter - +- CometNativeScan parquet spark_catalog.default.item + +- CometNativeColumnarToRow + +- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometNativeColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.store + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.customer + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.customer_address + +- CometBroadcastExchange + +- CometProject + +- CometFilter + +- CometNativeScan parquet spark_catalog.default.item -Comet accelerated 36 out of 83 eligible operators (43%). Final plan contains 16 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 80 out of 87 eligible operators (91%). Final plan contains 4 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q63.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q63.native_datafusion/extended.txt index 5c7a26057d..936cde052f 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q63.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q63.native_datafusion/extended.txt @@ -4,39 +4,34 @@ TakeOrderedAndProject +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] +- CometNativeColumnarToRow +- CometSort - +- CometColumnarExchange - +- HashAggregate - +- CometNativeColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.item - : : +- BroadcastExchange - : : +- Filter - : : +- ColumnarToRow - : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- CometNativeColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.date_dim - +- BroadcastExchange - +- CometNativeColumnarToRow - +- CometFilter - +- CometNativeScan parquet spark_catalog.default.store + +- CometExchange + +- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.item + : : +- CometBroadcastExchange + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : +- SubqueryBroadcast + : : +- BroadcastExchange + : : +- CometNativeColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.date_dim + +- CometBroadcastExchange + +- CometFilter + +- CometNativeScan parquet spark_catalog.default.store -Comet accelerated 14 out of 33 eligible operators (42%). Final plan contains 7 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 27 out of 33 eligible operators (81%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q64.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q64.native_datafusion/extended.txt index 98db64e5b0..f1fe6fb3e3 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q64.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q64.native_datafusion/extended.txt @@ -40,22 +40,20 @@ CometNativeColumnarToRow : : : : : : : : : : : : : : : :- CometProject : : : : : : : : : : : : : : : : +- CometSortMergeJoin : : : : : : : : : : : : : : : : :- CometSort - : : : : : : : : : : : : : : : : : +- CometColumnarExchange - : : : : : : : : : : : : : : : : : +- Project - : : : : : : : : : : : : : : : : : +- BroadcastHashJoin - : : : : : : : : : : : : : : : : : :- BroadcastExchange - : : : : : : : : : : : : : : : : : : +- Filter - : : : : : : : : : : : : : : : : : : +- ColumnarToRow - : : : : : : : : : : : : : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : : : : : : : : : : : : : : +- SubqueryBroadcast - : : : : : : : : : : : : : : : : : : +- BroadcastExchange - : : : : : : : : : : : : : : : : : : +- CometNativeColumnarToRow - : : : : : : : : : : : : : : : : : : +- CometFilter - : : : : : : : : : : : : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : : : : : : : : : : : : : : +- CometNativeColumnarToRow - : : : : : : : : : : : : : : : : : +- CometProject - : : : : : : : : : : : : : : : : : +- CometFilter - : : : : : : : : : : : : : : : : : +- CometNativeScan parquet spark_catalog.default.store_returns + : : : : : : : : : : : : : : : : : +- CometExchange + : : : : : : : : : : : : : : : : : +- CometProject + : : : : : : : : : : : : : : : : : +- CometBroadcastHashJoin + : : : : : : : : : : : : : : : : : :- CometBroadcastExchange + : : : : : : : : : : : : : : : : : : +- CometFilter + : : : : : : : : : : : : : : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : : : : : : : : : : : : : : : +- SubqueryBroadcast + : : : : : : : : : : : : : : : : : : +- BroadcastExchange + : : : : : : : : : : : : : : : : : : +- CometNativeColumnarToRow + : : : : : : : : : : : : : : : : : : +- CometFilter + : : : : : : : : : : : : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : : : : : : : : : : : : +- CometProject + : : : : : : : : : : : : : : : : : +- CometFilter + : : : : : : : : : : : : : : : : : +- CometNativeScan parquet spark_catalog.default.store_returns : : : : : : : : : : : : : : : : +- CometSort : : : : : : : : : : : : : : : : +- CometProject : : : : : : : : : : : : : : : : +- CometFilter @@ -162,22 +160,20 @@ CometNativeColumnarToRow : : : : : : : : : : : : : : :- CometProject : : : : : : : : : : : : : : : +- CometSortMergeJoin : : : : : : : : : : : : : : : :- CometSort - : : : : : : : : : : : : : : : : +- CometColumnarExchange - : : : : : : : : : : : : : : : : +- Project - : : : : : : : : : : : : : : : : +- BroadcastHashJoin - : : : : : : : : : : : : : : : : :- BroadcastExchange - : : : : : : : : : : : : : : : : : +- Filter - : : : : : : : : : : : : : : : : : +- ColumnarToRow - : : : : : : : : : : : : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : : : : : : : : : : : : : +- SubqueryBroadcast - : : : : : : : : : : : : : : : : : +- BroadcastExchange - : : : : : : : : : : : : : : : : : +- CometNativeColumnarToRow - : : : : : : : : : : : : : : : : : +- CometFilter - : : : : : : : : : : : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : : : : : : : : : : : : : +- CometNativeColumnarToRow - : : : : : : : : : : : : : : : : +- CometProject - : : : : : : : : : : : : : : : : +- CometFilter - : : : : : : : : : : : : : : : : +- CometNativeScan parquet spark_catalog.default.store_returns + : : : : : : : : : : : : : : : : +- CometExchange + : : : : : : : : : : : : : : : : +- CometProject + : : : : : : : : : : : : : : : : +- CometBroadcastHashJoin + : : : : : : : : : : : : : : : : :- CometBroadcastExchange + : : : : : : : : : : : : : : : : : +- CometFilter + : : : : : : : : : : : : : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : : : : : : : : : : : : : : +- SubqueryBroadcast + : : : : : : : : : : : : : : : : : +- BroadcastExchange + : : : : : : : : : : : : : : : : : +- CometNativeColumnarToRow + : : : : : : : : : : : : : : : : : +- CometFilter + : : : : : : : : : : : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : : : : : : : : : : : +- CometProject + : : : : : : : : : : : : : : : : +- CometFilter + : : : : : : : : : : : : : : : : +- CometNativeScan parquet spark_catalog.default.store_returns : : : : : : : : : : : : : : : +- CometSort : : : : : : : : : : : : : : : +- CometProject : : : : : : : : : : : : : : : +- CometFilter @@ -248,4 +244,4 @@ CometNativeColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.item -Comet accelerated 228 out of 242 eligible operators (94%). Final plan contains 7 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 238 out of 242 eligible operators (98%). Final plan contains 3 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q65.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q65.native_datafusion/extended.txt index 73e3aa7046..167a3386c2 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q65.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q65.native_datafusion/extended.txt @@ -1,60 +1,57 @@ -TakeOrderedAndProject -+- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- CometNativeColumnarToRow - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.store - : : +- BroadcastExchange - : : +- Filter - : : +- HashAggregate - : : +- CometNativeColumnarToRow - : : +- CometColumnarExchange - : : +- HashAggregate - : : +- Project - : : +- BroadcastHashJoin - : : :- Filter - : : : +- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- CometNativeColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.item - +- BroadcastExchange - +- Filter - +- HashAggregate - +- CometNativeColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- HashAggregate - +- CometNativeColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Project - +- BroadcastHashJoin - :- Filter - : +- ColumnarToRow - : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : +- ReusedSubquery - +- BroadcastExchange - +- CometNativeColumnarToRow - +- CometProject - +- CometFilter - +- CometNativeScan parquet spark_catalog.default.date_dim +CometNativeColumnarToRow ++- CometTakeOrderedAndProject + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.store + : : +- CometBroadcastExchange + : : +- CometFilter + : : +- CometHashAggregate + : : +- CometExchange + : : +- CometHashAggregate + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometNativeColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.item + +- CometBroadcastExchange + +- CometFilter + +- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometProject + +- CometBroadcastHashJoin + :- CometFilter + : +- CometNativeScan parquet spark_catalog.default.store_sales + : +- SubqueryBroadcast + : +- BroadcastExchange + : +- CometNativeColumnarToRow + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.date_dim + +- CometBroadcastExchange + +- CometProject + +- CometFilter + +- CometNativeScan parquet spark_catalog.default.date_dim -Comet accelerated 17 out of 48 eligible operators (35%). Final plan contains 10 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 48 out of 52 eligible operators (92%). Final plan contains 3 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q66.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q66.native_datafusion/extended.txt index 3f3c6b8d79..a7423e3a59 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q66.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q66.native_datafusion/extended.txt @@ -1,82 +1,74 @@ -TakeOrderedAndProject -+- HashAggregate - +- CometNativeColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Union - :- HashAggregate - : +- CometNativeColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- Project - : : : : +- BroadcastHashJoin - : : : : :- Filter - : : : : : +- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.warehouse - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.time_dim - : +- BroadcastExchange - : +- CometNativeColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.ship_mode - +- HashAggregate - +- CometNativeColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- Filter - : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- ReusedSubquery - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.warehouse - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- CometNativeColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.time_dim - +- BroadcastExchange - +- CometNativeColumnarToRow - +- CometProject - +- CometFilter - +- CometNativeScan parquet spark_catalog.default.ship_mode +CometNativeColumnarToRow ++- CometTakeOrderedAndProject + +- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometUnion + :- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.web_sales + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometNativeColumnarToRow + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.warehouse + : : : +- CometBroadcastExchange + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.time_dim + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.ship_mode + +- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometNativeColumnarToRow + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.warehouse + : : +- CometBroadcastExchange + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.time_dim + +- CometBroadcastExchange + +- CometProject + +- CometFilter + +- CometNativeScan parquet spark_catalog.default.ship_mode -Comet accelerated 27 out of 66 eligible operators (40%). Final plan contains 14 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 65 out of 69 eligible operators (94%). Final plan contains 3 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q67.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q67.native_datafusion/extended.txt index 27b7a8ecb4..892f37cf2d 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q67.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q67.native_datafusion/extended.txt @@ -3,41 +3,36 @@ TakeOrderedAndProject +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] +- CometNativeColumnarToRow +- CometSort - +- CometColumnarExchange - +- HashAggregate - +- CometNativeColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Expand - +- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Filter - : : : +- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- CometNativeColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.store - +- BroadcastExchange - +- CometNativeColumnarToRow - +- CometProject - +- CometFilter - +- CometNativeScan parquet spark_catalog.default.item + +- CometExchange + +- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometExpand + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometNativeColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.store + +- CometBroadcastExchange + +- CometProject + +- CometFilter + +- CometNativeScan parquet spark_catalog.default.item -Comet accelerated 15 out of 34 eligible operators (44%). Final plan contains 7 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 29 out of 34 eligible operators (85%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q68.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q68.native_datafusion/extended.txt index cebcd85144..dbdaaa275a 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q68.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q68.native_datafusion/extended.txt @@ -1,56 +1,49 @@ -TakeOrderedAndProject -+- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- HashAggregate - : : +- CometNativeColumnarToRow - : : +- CometColumnarExchange - : : +- HashAggregate - : : +- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- Project - : : : : +- BroadcastHashJoin - : : : : :- Project - : : : : : +- BroadcastHashJoin - : : : : : :- Filter - : : : : : : +- ColumnarToRow - : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometNativeColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : : +- BroadcastExchange - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.store - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.household_demographics - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.customer_address - : +- BroadcastExchange - : +- CometNativeColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.customer - +- BroadcastExchange - +- CometNativeColumnarToRow +CometNativeColumnarToRow ++- CometTakeOrderedAndProject + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometHashAggregate + : : +- CometExchange + : : +- CometHashAggregate + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometProject + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : : : +- SubqueryBroadcast + : : : : : : +- BroadcastExchange + : : : : : : +- CometNativeColumnarToRow + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- CometBroadcastExchange + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.store + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.household_demographics + : : +- CometBroadcastExchange + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.customer_address + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.customer + +- CometBroadcastExchange +- CometFilter +- CometNativeScan parquet spark_catalog.default.customer_address -Comet accelerated 20 out of 45 eligible operators (44%). Final plan contains 9 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 43 out of 45 eligible operators (95%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q69.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q69.native_datafusion/extended.txt index f48c67d6e1..3e0ba48add 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q69.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q69.native_datafusion/extended.txt @@ -9,46 +9,52 @@ TakeOrderedAndProject : +- BroadcastHashJoin : :- Project : : +- BroadcastHashJoin - : : :- BroadcastHashJoin - : : : :- BroadcastHashJoin - : : : : :- CometNativeColumnarToRow - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.customer - : : : : +- BroadcastExchange - : : : : +- Project - : : : : +- BroadcastHashJoin - : : : : :- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow + : : :- BroadcastHashJoin [COMET: BuildRight with LeftAnti is not supported] + : : : :- CometNativeColumnarToRow + : : : : +- CometBroadcastHashJoin + : : : : :- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.customer + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometNativeScan parquet spark_catalog.default.store_sales + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometNativeColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange : : : : +- CometProject : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- BroadcastExchange - : : : +- Project - : : : +- BroadcastHashJoin - : : : :- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- ReusedSubquery - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow + : : : +- CometNativeColumnarToRow + : : : +- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometNativeScan parquet spark_catalog.default.web_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometNativeColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- BroadcastExchange - : : +- Project - : : +- BroadcastHashJoin - : : :- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : +- ReusedSubquery - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow + : : +- CometNativeColumnarToRow + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometNativeColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange : : +- CometProject : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -63,4 +69,4 @@ TakeOrderedAndProject +- CometFilter +- CometNativeScan parquet spark_catalog.default.customer_demographics -Comet accelerated 21 out of 53 eligible operators (39%). Final plan contains 11 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 41 out of 61 eligible operators (67%). Final plan contains 9 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q7.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q7.native_datafusion/extended.txt index b692dfa9be..42206d1be5 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q7.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q7.native_datafusion/extended.txt @@ -1,44 +1,39 @@ -TakeOrderedAndProject -+- HashAggregate - +- CometNativeColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- Filter - : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.customer_demographics - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- CometNativeColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.item - +- BroadcastExchange - +- CometNativeColumnarToRow - +- CometProject - +- CometFilter - +- CometNativeScan parquet spark_catalog.default.promotion +CometNativeColumnarToRow ++- CometTakeOrderedAndProject + +- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometNativeColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.customer_demographics + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.item + +- CometBroadcastExchange + +- CometProject + +- CometFilter + +- CometNativeScan parquet spark_catalog.default.promotion -Comet accelerated 16 out of 35 eligible operators (45%). Final plan contains 7 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 33 out of 35 eligible operators (94%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q70.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q70.native_datafusion/extended.txt index 546d2cc0bc..7c3d343454 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q70.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q70.native_datafusion/extended.txt @@ -11,19 +11,18 @@ TakeOrderedAndProject +- Expand +- Project +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- Filter - : : +- ColumnarToRow - : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- CometNativeColumnarToRow + :- CometNativeColumnarToRow + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : +- SubqueryBroadcast + : : +- BroadcastExchange + : : +- CometNativeColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange : +- CometProject : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -36,29 +35,31 @@ TakeOrderedAndProject +- BroadcastExchange +- Project +- Filter - +- Window - +- Sort - +- HashAggregate - +- CometNativeColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- Filter - : : +- ColumnarToRow - : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : +- ReusedSubquery - : +- BroadcastExchange - : +- CometNativeColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.store - +- BroadcastExchange - +- CometNativeColumnarToRow - +- CometProject - +- CometFilter - +- CometNativeScan parquet spark_catalog.default.date_dim + +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] + +- CometNativeColumnarToRow + +- CometSort + +- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : +- SubqueryBroadcast + : : +- BroadcastExchange + : : +- CometNativeColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.store + +- CometBroadcastExchange + +- CometProject + +- CometFilter + +- CometNativeScan parquet spark_catalog.default.date_dim -Comet accelerated 18 out of 52 eligible operators (34%). Final plan contains 10 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 37 out of 56 eligible operators (66%). Final plan contains 7 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q71.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q71.native_datafusion/extended.txt index 23844f41f3..7f76dd192d 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q71.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q71.native_datafusion/extended.txt @@ -1,62 +1,63 @@ CometNativeColumnarToRow +- CometSort - +- CometColumnarExchange - +- HashAggregate - +- CometNativeColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.item - : +- Union - : :- Project - : : +- BroadcastHashJoin - : : :- Filter - : : : +- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : :- Project - : : +- BroadcastHashJoin - : : :- Filter - : : : +- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : +- ReusedSubquery - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- Project - : +- BroadcastHashJoin - : :- Filter - : : +- ColumnarToRow - : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : +- ReusedSubquery - : +- BroadcastExchange - : +- CometNativeColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.date_dim - +- BroadcastExchange - +- CometNativeColumnarToRow - +- CometProject - +- CometFilter - +- CometNativeScan parquet spark_catalog.default.time_dim + +- CometExchange + +- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.item + : +- CometUnion + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.web_sales + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometNativeColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometNativeColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : +- SubqueryBroadcast + : : +- BroadcastExchange + : : +- CometNativeColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.date_dim + +- CometBroadcastExchange + +- CometProject + +- CometFilter + +- CometNativeScan parquet spark_catalog.default.time_dim -Comet accelerated 21 out of 49 eligible operators (42%). Final plan contains 11 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 51 out of 57 eligible operators (89%). Final plan contains 4 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q72.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q72.native_datafusion/extended.txt index 62c2b17b7e..623a68ddab 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q72.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q72.native_datafusion/extended.txt @@ -6,77 +6,67 @@ CometNativeColumnarToRow +- CometProject +- CometSortMergeJoin :- CometSort - : +- CometColumnarExchange - : +- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- Project - : : : : +- BroadcastHashJoin - : : : : :- Project - : : : : : +- BroadcastHashJoin - : : : : : :- Project - : : : : : : +- BroadcastHashJoin - : : : : : : :- Project - : : : : : : : +- BroadcastHashJoin - : : : : : : : :- Project - : : : : : : : : +- BroadcastHashJoin - : : : : : : : : :- Project - : : : : : : : : : +- BroadcastHashJoin - : : : : : : : : : :- Filter - : : : : : : : : : : +- ColumnarToRow - : : : : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : : : : : : +- SubqueryBroadcast - : : : : : : : : : : +- BroadcastExchange - : : : : : : : : : : +- CometNativeColumnarToRow - : : : : : : : : : : +- CometProject - : : : : : : : : : : +- CometFilter - : : : : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : : : : : : +- BroadcastExchange - : : : : : : : : : +- CometNativeColumnarToRow - : : : : : : : : : +- CometFilter - : : : : : : : : : +- CometNativeScan parquet spark_catalog.default.inventory - : : : : : : : : +- BroadcastExchange - : : : : : : : : +- CometNativeColumnarToRow - : : : : : : : : +- CometFilter - : : : : : : : : +- CometNativeScan parquet spark_catalog.default.warehouse - : : : : : : : +- BroadcastExchange - : : : : : : : +- CometNativeColumnarToRow - : : : : : : : +- CometFilter - : : : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : : : +- BroadcastExchange - : : : : : : +- CometNativeColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.customer_demographics - : : : : : +- BroadcastExchange - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.household_demographics - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- CometNativeColumnarToRow - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.promotion + : +- CometExchange + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometProject + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometProject + : : : : : : +- CometBroadcastHashJoin + : : : : : : :- CometProject + : : : : : : : +- CometBroadcastHashJoin + : : : : : : : :- CometProject + : : : : : : : : +- CometBroadcastHashJoin + : : : : : : : : :- CometProject + : : : : : : : : : +- CometBroadcastHashJoin + : : : : : : : : : :- CometFilter + : : : : : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : : : : : : : +- SubqueryBroadcast + : : : : : : : : : : +- BroadcastExchange + : : : : : : : : : : +- CometNativeColumnarToRow + : : : : : : : : : : +- CometProject + : : : : : : : : : : +- CometFilter + : : : : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : : : : +- CometBroadcastExchange + : : : : : : : : : +- CometFilter + : : : : : : : : : +- CometNativeScan parquet spark_catalog.default.inventory + : : : : : : : : +- CometBroadcastExchange + : : : : : : : : +- CometFilter + : : : : : : : : +- CometNativeScan parquet spark_catalog.default.warehouse + : : : : : : : +- CometBroadcastExchange + : : : : : : : +- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : : : : +- CometBroadcastExchange + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.customer_demographics + : : : : : +- CometBroadcastExchange + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.household_demographics + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.promotion +- CometSort +- CometExchange +- CometProject +- CometFilter +- CometNativeScan parquet spark_catalog.default.catalog_returns -Comet accelerated 37 out of 68 eligible operators (54%). Final plan contains 12 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 66 out of 68 eligible operators (97%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q73.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q73.native_datafusion/extended.txt index 40230582bb..1b61be2937 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q73.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q73.native_datafusion/extended.txt @@ -1,47 +1,41 @@ CometNativeColumnarToRow +- CometSort - +- CometColumnarExchange - +- Project - +- BroadcastHashJoin - :- Filter - : +- HashAggregate - : +- CometNativeColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- Filter - : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.store - : +- BroadcastExchange - : +- CometNativeColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.household_demographics - +- BroadcastExchange - +- CometNativeColumnarToRow - +- CometProject - +- CometFilter - +- CometNativeScan parquet spark_catalog.default.customer + +- CometExchange + +- CometProject + +- CometBroadcastHashJoin + :- CometFilter + : +- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometNativeColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.store + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.household_demographics + +- CometBroadcastExchange + +- CometProject + +- CometFilter + +- CometNativeScan parquet spark_catalog.default.customer -Comet accelerated 18 out of 37 eligible operators (48%). Final plan contains 8 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 35 out of 37 eligible operators (94%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q74.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q74.native_datafusion/extended.txt index 5d218998be..51a3f5ca94 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q74.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q74.native_datafusion/extended.txt @@ -1,105 +1,98 @@ -TakeOrderedAndProject -+- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- BroadcastHashJoin - : : :- Filter - : : : +- HashAggregate - : : : +- CometNativeColumnarToRow - : : : +- CometColumnarExchange - : : : +- HashAggregate - : : : +- Project - : : : +- BroadcastHashJoin - : : : :- Project - : : : : +- BroadcastHashJoin - : : : : :- CometNativeColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.customer - : : : : +- BroadcastExchange - : : : : +- Filter - : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- HashAggregate - : : +- CometNativeColumnarToRow - : : +- CometColumnarExchange - : : +- HashAggregate - : : +- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.customer - : : : +- BroadcastExchange - : : : +- Filter - : : : +- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- Filter - : +- HashAggregate - : +- CometNativeColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.customer - : : +- BroadcastExchange - : : +- Filter - : : +- ColumnarToRow - : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : +- ReusedSubquery - : +- BroadcastExchange - : +- CometNativeColumnarToRow - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.date_dim - +- BroadcastExchange - +- HashAggregate - +- CometNativeColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- CometNativeColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.customer - : +- BroadcastExchange - : +- Filter - : +- ColumnarToRow - : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : +- ReusedSubquery - +- BroadcastExchange - +- CometNativeColumnarToRow - +- CometFilter - +- CometNativeScan parquet spark_catalog.default.date_dim +CometNativeColumnarToRow ++- CometTakeOrderedAndProject + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometHashAggregate + : : : +- CometExchange + : : : +- CometHashAggregate + : : : +- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.customer + : : : : +- CometBroadcastExchange + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometNativeColumnarToRow + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometHashAggregate + : : +- CometExchange + : : +- CometHashAggregate + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.customer + : : : +- CometBroadcastExchange + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometNativeColumnarToRow + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometFilter + : +- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.customer + : : +- CometBroadcastExchange + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.web_sales + : : +- SubqueryBroadcast + : : +- BroadcastExchange + : : +- CometNativeColumnarToRow + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.date_dim + +- CometBroadcastExchange + +- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.customer + : +- CometBroadcastExchange + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.web_sales + : +- SubqueryBroadcast + : +- BroadcastExchange + : +- CometNativeColumnarToRow + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.date_dim + +- CometBroadcastExchange + +- CometFilter + +- CometNativeScan parquet spark_catalog.default.date_dim -Comet accelerated 28 out of 85 eligible operators (32%). Final plan contains 18 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 83 out of 91 eligible operators (91%). Final plan contains 5 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q75.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q75.native_datafusion/extended.txt index 42106a0ffd..a943308f1e 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q75.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q75.native_datafusion/extended.txt @@ -15,28 +15,25 @@ CometNativeColumnarToRow : :- CometProject : : +- CometSortMergeJoin : : :- CometSort - : : : +- CometColumnarExchange - : : : +- Project - : : : +- BroadcastHashJoin - : : : :- Project - : : : : +- BroadcastHashJoin - : : : : :- Filter - : : : : : +- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometExchange + : : : +- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometNativeColumnarToRow + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : +- CometBroadcastExchange + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- CometSort : : +- CometExchange : : +- CometProject @@ -45,24 +42,25 @@ CometNativeColumnarToRow : :- CometProject : : +- CometSortMergeJoin : : :- CometSort - : : : +- CometColumnarExchange - : : : +- Project - : : : +- BroadcastHashJoin - : : : :- Project - : : : : +- BroadcastHashJoin - : : : : :- Filter - : : : : : +- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : +- ReusedSubquery - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometExchange + : : : +- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometNativeColumnarToRow + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : +- CometBroadcastExchange + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- CometSort : : +- CometExchange : : +- CometProject @@ -71,24 +69,25 @@ CometNativeColumnarToRow : +- CometProject : +- CometSortMergeJoin : :- CometSort - : : +- CometColumnarExchange - : : +- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- Filter - : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- ReusedSubquery - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.item - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometExchange + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.web_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometNativeColumnarToRow + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.item + : : +- CometBroadcastExchange + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim : +- CometSort : +- CometExchange : +- CometProject @@ -107,28 +106,25 @@ CometNativeColumnarToRow :- CometProject : +- CometSortMergeJoin : :- CometSort - : : +- CometColumnarExchange - : : +- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- Filter - : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.item - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometExchange + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometNativeColumnarToRow + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.item + : : +- CometBroadcastExchange + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim : +- CometSort : +- CometExchange : +- CometProject @@ -137,24 +133,25 @@ CometNativeColumnarToRow :- CometProject : +- CometSortMergeJoin : :- CometSort - : : +- CometColumnarExchange - : : +- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- Filter - : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- ReusedSubquery - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.item - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometExchange + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometNativeColumnarToRow + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.item + : : +- CometBroadcastExchange + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim : +- CometSort : +- CometExchange : +- CometProject @@ -163,28 +160,29 @@ CometNativeColumnarToRow +- CometProject +- CometSortMergeJoin :- CometSort - : +- CometColumnarExchange - : +- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Filter - : : : +- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : +- ReusedSubquery - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.item - : +- BroadcastExchange - : +- CometNativeColumnarToRow - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometExchange + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.web_sales + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometNativeColumnarToRow + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.item + : +- CometBroadcastExchange + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.date_dim +- CometSort +- CometExchange +- CometProject +- CometFilter +- CometNativeScan parquet spark_catalog.default.web_returns -Comet accelerated 111 out of 167 eligible operators (66%). Final plan contains 21 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 167 out of 179 eligible operators (93%). Final plan contains 7 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q77.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q77.native_datafusion/extended.txt index 5d2ca1a9d0..835f7a449a 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q77.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q77.native_datafusion/extended.txt @@ -5,131 +5,138 @@ TakeOrderedAndProject +- HashAggregate +- Expand +- Union + :- CometNativeColumnarToRow + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometHashAggregate + : : +- CometExchange + : : +- CometHashAggregate + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometNativeColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.store + : +- CometBroadcastExchange + : +- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.store_returns + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometNativeColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.store :- Project - : +- BroadcastHashJoin - : :- HashAggregate - : : +- CometNativeColumnarToRow - : : +- CometColumnarExchange - : : +- HashAggregate - : : +- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- Filter - : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.store - : +- BroadcastExchange - : +- HashAggregate - : +- CometNativeColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Filter - : : : +- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.store_returns [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : +- ReusedSubquery - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- CometNativeColumnarToRow - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.store - :- Project - : +- BroadcastNestedLoopJoin + : +- BroadcastNestedLoopJoin [COMET: BroadcastNestedLoopJoin is not supported] : :- BroadcastExchange - : : +- HashAggregate - : : +- CometNativeColumnarToRow - : : +- CometColumnarExchange - : : +- HashAggregate - : : +- Project - : : +- BroadcastHashJoin - : : :- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : +- ReusedSubquery - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- HashAggregate - : +- CometNativeColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- ColumnarToRow - : : +- Scan parquet spark_catalog.default.catalog_returns [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : +- ReusedSubquery - : +- BroadcastExchange - : +- CometNativeColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.date_dim - +- Project - +- BroadcastHashJoin - :- HashAggregate - : +- CometNativeColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Filter - : : : +- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : +- ReusedSubquery - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- CometNativeColumnarToRow - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.web_page - +- BroadcastExchange - +- HashAggregate - +- CometNativeColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- Filter - : : +- ColumnarToRow - : : +- Scan parquet spark_catalog.default.web_returns [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : +- ReusedSubquery - : +- BroadcastExchange - : +- CometNativeColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.date_dim - +- BroadcastExchange - +- CometNativeColumnarToRow - +- CometFilter - +- CometNativeScan parquet spark_catalog.default.web_page + : : +- CometNativeColumnarToRow + : : +- CometHashAggregate + : : +- CometExchange + : : +- CometHashAggregate + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometNativeColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometNativeColumnarToRow + : +- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometNativeScan parquet spark_catalog.default.catalog_returns + : : +- SubqueryBroadcast + : : +- BroadcastExchange + : : +- CometNativeColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.date_dim + +- CometNativeColumnarToRow + +- CometProject + +- CometBroadcastHashJoin + :- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.web_sales + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometNativeColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.web_page + +- CometBroadcastExchange + +- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.web_returns + : : +- SubqueryBroadcast + : : +- BroadcastExchange + : : +- CometNativeColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.date_dim + +- CometBroadcastExchange + +- CometFilter + +- CometNativeScan parquet spark_catalog.default.web_page -Comet accelerated 36 out of 109 eligible operators (33%). Final plan contains 24 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 109 out of 129 eligible operators (84%). Final plan contains 11 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q78.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q78.native_datafusion/extended.txt index af4b7000b9..ebfdcc3c62 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q78.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q78.native_datafusion/extended.txt @@ -14,15 +14,14 @@ TakeOrderedAndProject : : : +- CometFilter : : : +- CometSortMergeJoin : : : :- CometSort - : : : : +- CometColumnarExchange - : : : : +- Filter - : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometExchange + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometNativeColumnarToRow + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- CometSort : : : +- CometExchange : : : +- CometProject @@ -42,11 +41,14 @@ TakeOrderedAndProject : : +- CometFilter : : +- CometSortMergeJoin : : :- CometSort - : : : +- CometColumnarExchange - : : : +- Filter - : : : +- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : +- ReusedSubquery + : : : +- CometExchange + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.web_sales + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometNativeColumnarToRow + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- CometSort : : +- CometExchange : : +- CometProject @@ -66,11 +68,14 @@ TakeOrderedAndProject : +- CometFilter : +- CometSortMergeJoin : :- CometSort - : : +- CometColumnarExchange - : : +- Filter - : : +- ColumnarToRow - : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : +- ReusedSubquery + : : +- CometExchange + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : +- SubqueryBroadcast + : : +- BroadcastExchange + : : +- CometNativeColumnarToRow + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim : +- CometSort : +- CometExchange : +- CometProject @@ -80,4 +85,4 @@ TakeOrderedAndProject +- CometFilter +- CometNativeScan parquet spark_catalog.default.date_dim -Comet accelerated 64 out of 76 eligible operators (84%). Final plan contains 5 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 74 out of 82 eligible operators (90%). Final plan contains 4 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q79.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q79.native_datafusion/extended.txt index f24d730d8b..526b86f475 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q79.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q79.native_datafusion/extended.txt @@ -1,44 +1,39 @@ -TakeOrderedAndProject -+- Project - +- BroadcastHashJoin - :- HashAggregate - : +- CometNativeColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- Filter - : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.store - : +- BroadcastExchange - : +- CometNativeColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.household_demographics - +- BroadcastExchange - +- CometNativeColumnarToRow +CometNativeColumnarToRow ++- CometTakeOrderedAndProject + +- CometProject + +- CometBroadcastHashJoin + :- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometNativeColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.store + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.household_demographics + +- CometBroadcastExchange +- CometProject +- CometFilter +- CometNativeScan parquet spark_catalog.default.customer -Comet accelerated 16 out of 35 eligible operators (45%). Final plan contains 7 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 33 out of 35 eligible operators (94%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q8.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q8.native_datafusion/extended.txt index 8c9eb7b8c9..cb04fc12c8 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q8.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q8.native_datafusion/extended.txt @@ -1,56 +1,52 @@ -TakeOrderedAndProject -+- HashAggregate - +- CometNativeColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Filter - : : : +- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- CometNativeColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.store - +- BroadcastExchange - +- CometNativeColumnarToRow - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometBroadcastHashJoin - :- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.customer_address - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.customer_address - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometNativeScan parquet spark_catalog.default.customer +CometNativeColumnarToRow ++- CometTakeOrderedAndProject + +- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometNativeColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.store + +- CometBroadcastExchange + +- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometBroadcastHashJoin + :- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.customer_address + +- CometBroadcastExchange + +- CometProject + +- CometFilter + +- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.customer_address + +- CometBroadcastExchange + +- CometProject + +- CometFilter + +- CometNativeScan parquet spark_catalog.default.customer -Comet accelerated 32 out of 48 eligible operators (66%). Final plan contains 6 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 46 out of 48 eligible operators (95%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q80.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q80.native_datafusion/extended.txt index 6853afccad..003be45ee5 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q80.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q80.native_datafusion/extended.txt @@ -19,16 +19,15 @@ CometNativeColumnarToRow : : : : :- CometProject : : : : : +- CometSortMergeJoin : : : : : :- CometSort - : : : : : : +- CometColumnarExchange - : : : : : : +- Filter - : : : : : : +- ColumnarToRow - : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometNativeColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : +- CometExchange + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : : : +- SubqueryBroadcast + : : : : : : +- BroadcastExchange + : : : : : : +- CometNativeColumnarToRow + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : : +- CometSort : : : : : +- CometExchange : : : : : +- CometProject @@ -64,11 +63,15 @@ CometNativeColumnarToRow : : : : :- CometProject : : : : : +- CometSortMergeJoin : : : : : :- CometSort - : : : : : : +- CometColumnarExchange - : : : : : : +- Filter - : : : : : : +- ColumnarToRow - : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : : +- ReusedSubquery + : : : : : : +- CometExchange + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : : : +- SubqueryBroadcast + : : : : : : +- BroadcastExchange + : : : : : : +- CometNativeColumnarToRow + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : : +- CometSort : : : : : +- CometExchange : : : : : +- CometProject @@ -104,11 +107,15 @@ CometNativeColumnarToRow : : : :- CometProject : : : : +- CometSortMergeJoin : : : : :- CometSort - : : : : : +- CometColumnarExchange - : : : : : +- Filter - : : : : : +- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : +- ReusedSubquery + : : : : : +- CometExchange + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.web_sales + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometNativeColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : +- CometSort : : : : +- CometExchange : : : : +- CometProject @@ -131,4 +138,4 @@ CometNativeColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.promotion -Comet accelerated 117 out of 127 eligible operators (92%). Final plan contains 5 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 129 out of 135 eligible operators (95%). Final plan contains 4 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q81.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q81.native_datafusion/extended.txt index 841b2ca52d..949d50a867 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q81.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q81.native_datafusion/extended.txt @@ -1,75 +1,70 @@ -TakeOrderedAndProject -+- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Filter - : : : +- HashAggregate - : : : +- CometNativeColumnarToRow - : : : +- CometColumnarExchange - : : : +- HashAggregate - : : : +- Project - : : : +- BroadcastHashJoin - : : : :- Project - : : : : +- BroadcastHashJoin - : : : : :- Filter - : : : : : +- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.catalog_returns [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.customer_address - : : +- BroadcastExchange - : : +- Filter - : : +- HashAggregate - : : +- CometNativeColumnarToRow - : : +- CometColumnarExchange - : : +- HashAggregate - : : +- HashAggregate - : : +- CometNativeColumnarToRow - : : +- CometColumnarExchange - : : +- HashAggregate - : : +- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- Filter - : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.catalog_returns [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- ReusedSubquery - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.customer_address - : +- BroadcastExchange - : +- CometNativeColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.customer - +- BroadcastExchange - +- CometNativeColumnarToRow +CometNativeColumnarToRow ++- CometTakeOrderedAndProject + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometHashAggregate + : : : +- CometExchange + : : : +- CometHashAggregate + : : : +- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_returns + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometNativeColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.customer_address + : : +- CometBroadcastExchange + : : +- CometFilter + : : +- CometHashAggregate + : : +- CometExchange + : : +- CometHashAggregate + : : +- CometHashAggregate + : : +- CometExchange + : : +- CometHashAggregate + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.catalog_returns + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometNativeColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.customer_address + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.customer + +- CometBroadcastExchange +- CometProject +- CometFilter +- CometNativeScan parquet spark_catalog.default.customer_address -Comet accelerated 24 out of 61 eligible operators (39%). Final plan contains 12 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 61 out of 65 eligible operators (93%). Final plan contains 3 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q82.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q82.native_datafusion/extended.txt index 8abf5c1fb1..30fca82ec8 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q82.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q82.native_datafusion/extended.txt @@ -1,38 +1,34 @@ CometNativeColumnarToRow +- CometTakeOrderedAndProject +- CometHashAggregate - +- CometColumnarExchange - +- HashAggregate - +- Project - +- BroadcastHashJoin - :- BroadcastExchange - : +- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.item - : : +- BroadcastExchange - : : +- Project - : : +- Filter - : : +- ColumnarToRow - : : +- Scan parquet spark_catalog.default.inventory [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- CometNativeColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.date_dim - +- CometNativeColumnarToRow - +- CometProject - +- CometFilter - +- CometNativeScan parquet spark_catalog.default.store_sales + +- CometExchange + +- CometHashAggregate + +- CometProject + +- CometBroadcastHashJoin + :- CometBroadcastExchange + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.item + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.inventory + : : +- SubqueryBroadcast + : : +- BroadcastExchange + : : +- CometNativeColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.date_dim + +- CometProject + +- CometFilter + +- CometNativeScan parquet spark_catalog.default.store_sales -Comet accelerated 15 out of 30 eligible operators (50%). Final plan contains 6 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 28 out of 30 eligible operators (93%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q83.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q83.native_datafusion/extended.txt index eac3126bae..3a16430014 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q83.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q83.native_datafusion/extended.txt @@ -1,130 +1,133 @@ -TakeOrderedAndProject -+- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- HashAggregate - : : +- CometNativeColumnarToRow - : : +- CometColumnarExchange - : : +- HashAggregate - : : +- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- Filter - : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.store_returns [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometNativeScan parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.item - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometNativeScan parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- HashAggregate - : +- CometNativeColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Filter - : : : +- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.catalog_returns [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.item - : +- BroadcastExchange - : +- CometNativeColumnarToRow - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometNativeScan parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.date_dim - +- BroadcastExchange - +- HashAggregate - +- CometNativeColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- Filter - : : +- ColumnarToRow - : : +- Scan parquet spark_catalog.default.web_returns [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : +- ReusedSubquery - : +- BroadcastExchange - : +- CometNativeColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.item - +- BroadcastExchange - +- CometNativeColumnarToRow - +- CometProject - +- CometBroadcastHashJoin - :- CometFilter - : +- CometNativeScan parquet spark_catalog.default.date_dim - +- CometBroadcastExchange - +- CometProject - +- CometBroadcastHashJoin - :- CometNativeScan parquet spark_catalog.default.date_dim - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometNativeScan parquet spark_catalog.default.date_dim +CometNativeColumnarToRow ++- CometTakeOrderedAndProject + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometHashAggregate + : : +- CometExchange + : : +- CometHashAggregate + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.store_returns + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometNativeColumnarToRow + : : : : +- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.item + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.catalog_returns + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometNativeColumnarToRow + : : : +- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.item + : +- CometBroadcastExchange + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.date_dim + +- CometBroadcastExchange + +- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.web_returns + : : +- SubqueryBroadcast + : : +- BroadcastExchange + : : +- CometNativeColumnarToRow + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.item + +- CometBroadcastExchange + +- CometProject + +- CometBroadcastHashJoin + :- CometFilter + : +- CometNativeScan parquet spark_catalog.default.date_dim + +- CometBroadcastExchange + +- CometProject + +- CometBroadcastHashJoin + :- CometNativeScan parquet spark_catalog.default.date_dim + +- CometBroadcastExchange + +- CometProject + +- CometFilter + +- CometNativeScan parquet spark_catalog.default.date_dim -Comet accelerated 72 out of 114 eligible operators (63%). Final plan contains 14 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 121 out of 127 eligible operators (95%). Final plan contains 4 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q85.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q85.native_datafusion/extended.txt index aed7b365be..06537a5eec 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q85.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q85.native_datafusion/extended.txt @@ -1,64 +1,56 @@ -TakeOrderedAndProject -+- HashAggregate - +- CometNativeColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- Project - : : : : +- BroadcastHashJoin - : : : : :- Project - : : : : : +- BroadcastHashJoin - : : : : : :- Project - : : : : : : +- BroadcastHashJoin - : : : : : : :- BroadcastExchange - : : : : : : : +- Filter - : : : : : : : +- ColumnarToRow - : : : : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : : : +- SubqueryBroadcast - : : : : : : : +- BroadcastExchange - : : : : : : : +- CometNativeColumnarToRow - : : : : : : : +- CometProject - : : : : : : : +- CometFilter - : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : : : +- CometNativeColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.web_returns - : : : : : +- BroadcastExchange - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.web_page - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.customer_demographics - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.customer_demographics - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.customer_address - : +- BroadcastExchange - : +- CometNativeColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.date_dim - +- BroadcastExchange - +- CometNativeColumnarToRow - +- CometProject - +- CometFilter - +- CometNativeScan parquet spark_catalog.default.reason +CometNativeColumnarToRow ++- CometTakeOrderedAndProject + +- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometProject + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometProject + : : : : : : +- CometBroadcastHashJoin + : : : : : : :- CometBroadcastExchange + : : : : : : : +- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.web_sales + : : : : : : : +- SubqueryBroadcast + : : : : : : : +- BroadcastExchange + : : : : : : : +- CometNativeColumnarToRow + : : : : : : : +- CometProject + : : : : : : : +- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.web_returns + : : : : : +- CometBroadcastExchange + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.web_page + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.customer_demographics + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.customer_demographics + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.customer_address + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.date_dim + +- CometBroadcastExchange + +- CometProject + +- CometFilter + +- CometNativeScan parquet spark_catalog.default.reason -Comet accelerated 24 out of 52 eligible operators (46%). Final plan contains 10 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 50 out of 52 eligible operators (96%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q86.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q86.native_datafusion/extended.txt index a60a3158a4..c9117e2ae0 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q86.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q86.native_datafusion/extended.txt @@ -3,34 +3,30 @@ TakeOrderedAndProject +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] +- CometNativeColumnarToRow +- CometSort - +- CometColumnarExchange - +- HashAggregate - +- CometNativeColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Expand - +- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- Filter - : : +- ColumnarToRow - : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- CometNativeColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.date_dim - +- BroadcastExchange - +- CometNativeColumnarToRow - +- CometProject - +- CometFilter - +- CometNativeScan parquet spark_catalog.default.item + +- CometExchange + +- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometExpand + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.web_sales + : : +- SubqueryBroadcast + : : +- BroadcastExchange + : : +- CometNativeColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.date_dim + +- CometBroadcastExchange + +- CometProject + +- CometFilter + +- CometNativeScan parquet spark_catalog.default.item -Comet accelerated 12 out of 28 eligible operators (42%). Final plan contains 6 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 23 out of 28 eligible operators (82%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q87.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q87.native_datafusion/extended.txt index 7a9e62d57c..6676ec7a4b 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q87.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q87.native_datafusion/extended.txt @@ -7,76 +7,77 @@ HashAggregate :- BroadcastHashJoin [COMET: BuildRight with LeftAnti is not supported] : :- CometNativeColumnarToRow : : +- CometHashAggregate - : : +- CometColumnarExchange - : : +- HashAggregate - : : +- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- Filter - : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.customer + : : +- CometExchange + : : +- CometHashAggregate + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometNativeColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.customer : +- BroadcastExchange : +- CometNativeColumnarToRow : +- CometHashAggregate - : +- CometColumnarExchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Filter - : : : +- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : +- ReusedSubquery - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- CometNativeColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.customer + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometNativeColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.customer +- BroadcastExchange +- CometNativeColumnarToRow +- CometHashAggregate - +- CometColumnarExchange - +- HashAggregate - +- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- Filter - : : +- ColumnarToRow - : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : +- ReusedSubquery - : +- BroadcastExchange - : +- CometNativeColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.date_dim - +- BroadcastExchange - +- CometNativeColumnarToRow - +- CometProject - +- CometFilter - +- CometNativeScan parquet spark_catalog.default.customer + +- CometExchange + +- CometHashAggregate + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.web_sales + : : +- SubqueryBroadcast + : : +- BroadcastExchange + : : +- CometNativeColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.date_dim + +- CometBroadcastExchange + +- CometProject + +- CometFilter + +- CometNativeScan parquet spark_catalog.default.customer -Comet accelerated 28 out of 66 eligible operators (42%). Final plan contains 14 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 61 out of 74 eligible operators (82%). Final plan contains 7 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q89.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q89.native_datafusion/extended.txt index 5c7a26057d..936cde052f 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q89.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q89.native_datafusion/extended.txt @@ -4,39 +4,34 @@ TakeOrderedAndProject +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] +- CometNativeColumnarToRow +- CometSort - +- CometColumnarExchange - +- HashAggregate - +- CometNativeColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.item - : : +- BroadcastExchange - : : +- Filter - : : +- ColumnarToRow - : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- CometNativeColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.date_dim - +- BroadcastExchange - +- CometNativeColumnarToRow - +- CometFilter - +- CometNativeScan parquet spark_catalog.default.store + +- CometExchange + +- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.item + : : +- CometBroadcastExchange + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : +- SubqueryBroadcast + : : +- BroadcastExchange + : : +- CometNativeColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.date_dim + +- CometBroadcastExchange + +- CometFilter + +- CometNativeScan parquet spark_catalog.default.store -Comet accelerated 14 out of 33 eligible operators (42%). Final plan contains 7 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 27 out of 33 eligible operators (81%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q91.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q91.native_datafusion/extended.txt index 6f982615ae..28bc6eab2f 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q91.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q91.native_datafusion/extended.txt @@ -1,59 +1,51 @@ CometNativeColumnarToRow +- CometSort - +- CometColumnarExchange - +- HashAggregate - +- CometNativeColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- Project - : : : : +- BroadcastHashJoin - : : : : :- Project - : : : : : +- BroadcastHashJoin - : : : : : :- CometNativeColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.call_center - : : : : : +- BroadcastExchange - : : : : : +- Filter - : : : : : +- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.catalog_returns [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.customer - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.customer_address - : +- BroadcastExchange - : +- CometNativeColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.customer_demographics - +- BroadcastExchange - +- CometNativeColumnarToRow - +- CometProject - +- CometFilter - +- CometNativeScan parquet spark_catalog.default.household_demographics + +- CometExchange + +- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometProject + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.call_center + : : : : : +- CometBroadcastExchange + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_returns + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometNativeColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.customer + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.customer_address + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.customer_demographics + +- CometBroadcastExchange + +- CometProject + +- CometFilter + +- CometNativeScan parquet spark_catalog.default.household_demographics -Comet accelerated 23 out of 47 eligible operators (48%). Final plan contains 10 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 45 out of 47 eligible operators (95%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q92.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q92.native_datafusion/extended.txt index b3ca85cdaf..6c5b57b3a1 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q92.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q92.native_datafusion/extended.txt @@ -1,48 +1,47 @@ -HashAggregate -+- CometNativeColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Filter - : : : +- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.item - : +- BroadcastExchange - : +- Filter - : +- HashAggregate - : +- CometNativeColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- Filter - : : +- ColumnarToRow - : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : +- ReusedSubquery - : +- BroadcastExchange - : +- CometNativeColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.date_dim - +- BroadcastExchange - +- CometNativeColumnarToRow - +- CometProject - +- CometFilter - +- CometNativeScan parquet spark_catalog.default.date_dim +CometNativeColumnarToRow ++- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.web_sales + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometNativeColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.item + : +- CometBroadcastExchange + : +- CometFilter + : +- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.web_sales + : : +- SubqueryBroadcast + : : +- BroadcastExchange + : : +- CometNativeColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.date_dim + +- CometBroadcastExchange + +- CometProject + +- CometFilter + +- CometNativeScan parquet spark_catalog.default.date_dim -Comet accelerated 14 out of 38 eligible operators (36%). Final plan contains 8 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 38 out of 42 eligible operators (90%). Final plan contains 3 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q97.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q97.native_datafusion/extended.txt index e3b777a05a..b63930a880 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q97.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q97.native_datafusion/extended.txt @@ -6,36 +6,37 @@ CometNativeColumnarToRow +- CometSortMergeJoin :- CometSort : +- CometHashAggregate - : +- CometColumnarExchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- ColumnarToRow - : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- CometNativeColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometNativeScan parquet spark_catalog.default.store_sales + : : +- SubqueryBroadcast + : : +- BroadcastExchange + : : +- CometNativeColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.date_dim +- CometSort +- CometHashAggregate - +- CometColumnarExchange - +- HashAggregate - +- Project - +- BroadcastHashJoin - :- ColumnarToRow - : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : +- ReusedSubquery - +- BroadcastExchange - +- CometNativeColumnarToRow - +- CometProject - +- CometFilter - +- CometNativeScan parquet spark_catalog.default.date_dim + +- CometExchange + +- CometHashAggregate + +- CometProject + +- CometBroadcastHashJoin + :- CometNativeScan parquet spark_catalog.default.catalog_sales + : +- SubqueryBroadcast + : +- BroadcastExchange + : +- CometNativeColumnarToRow + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.date_dim + +- CometBroadcastExchange + +- CometProject + +- CometFilter + +- CometNativeScan parquet spark_catalog.default.date_dim -Comet accelerated 20 out of 33 eligible operators (60%). Final plan contains 6 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 33 out of 37 eligible operators (89%). Final plan contains 3 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q98.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q98.native_datafusion/extended.txt index af8cf29205..6cc173560b 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q98.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q98.native_datafusion/extended.txt @@ -6,33 +6,29 @@ CometNativeColumnarToRow +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] +- CometNativeColumnarToRow +- CometSort - +- CometColumnarExchange - +- HashAggregate - +- CometNativeColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- Filter - : : +- ColumnarToRow - : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- CometNativeColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.item - +- BroadcastExchange - +- CometNativeColumnarToRow - +- CometProject - +- CometFilter - +- CometNativeScan parquet spark_catalog.default.date_dim + +- CometExchange + +- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : +- SubqueryBroadcast + : : +- BroadcastExchange + : : +- CometNativeColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.item + +- CometBroadcastExchange + +- CometProject + +- CometFilter + +- CometNativeScan parquet spark_catalog.default.date_dim -Comet accelerated 15 out of 29 eligible operators (51%). Final plan contains 7 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 25 out of 29 eligible operators (86%). Final plan contains 3 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q10a.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q10a.native_datafusion/extended.txt index 9091536e96..cba125f412 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q10a.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q10a.native_datafusion/extended.txt @@ -1,65 +1,66 @@ -TakeOrderedAndProject -+- HashAggregate - +- CometNativeColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- BroadcastHashJoin - : : : :- CometNativeColumnarToRow - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.customer - : : : +- BroadcastExchange - : : : +- Project - : : : +- BroadcastHashJoin - : : : :- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- Union - : : :- Project - : : : +- BroadcastHashJoin - : : : :- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- ReusedSubquery - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- Project - : : +- BroadcastHashJoin - : : :- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : +- ReusedSubquery - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- CometNativeColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.customer_address - +- BroadcastExchange - +- CometNativeColumnarToRow - +- CometProject - +- CometFilter - +- CometNativeScan parquet spark_catalog.default.customer_demographics +CometNativeColumnarToRow ++- CometTakeOrderedAndProject + +- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.customer + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometNativeScan parquet spark_catalog.default.store_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometNativeColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometUnion + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometNativeScan parquet spark_catalog.default.web_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometNativeColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometNativeColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.customer_address + +- CometBroadcastExchange + +- CometProject + +- CometFilter + +- CometNativeScan parquet spark_catalog.default.customer_demographics -Comet accelerated 21 out of 52 eligible operators (40%). Final plan contains 11 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 54 out of 60 eligible operators (90%). Final plan contains 4 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q11.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q11.native_datafusion/extended.txt index 5d218998be..51a3f5ca94 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q11.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q11.native_datafusion/extended.txt @@ -1,105 +1,98 @@ -TakeOrderedAndProject -+- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- BroadcastHashJoin - : : :- Filter - : : : +- HashAggregate - : : : +- CometNativeColumnarToRow - : : : +- CometColumnarExchange - : : : +- HashAggregate - : : : +- Project - : : : +- BroadcastHashJoin - : : : :- Project - : : : : +- BroadcastHashJoin - : : : : :- CometNativeColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.customer - : : : : +- BroadcastExchange - : : : : +- Filter - : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- HashAggregate - : : +- CometNativeColumnarToRow - : : +- CometColumnarExchange - : : +- HashAggregate - : : +- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.customer - : : : +- BroadcastExchange - : : : +- Filter - : : : +- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- Filter - : +- HashAggregate - : +- CometNativeColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.customer - : : +- BroadcastExchange - : : +- Filter - : : +- ColumnarToRow - : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : +- ReusedSubquery - : +- BroadcastExchange - : +- CometNativeColumnarToRow - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.date_dim - +- BroadcastExchange - +- HashAggregate - +- CometNativeColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- CometNativeColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.customer - : +- BroadcastExchange - : +- Filter - : +- ColumnarToRow - : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : +- ReusedSubquery - +- BroadcastExchange - +- CometNativeColumnarToRow - +- CometFilter - +- CometNativeScan parquet spark_catalog.default.date_dim +CometNativeColumnarToRow ++- CometTakeOrderedAndProject + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometHashAggregate + : : : +- CometExchange + : : : +- CometHashAggregate + : : : +- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.customer + : : : : +- CometBroadcastExchange + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometNativeColumnarToRow + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometHashAggregate + : : +- CometExchange + : : +- CometHashAggregate + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.customer + : : : +- CometBroadcastExchange + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometNativeColumnarToRow + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometFilter + : +- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.customer + : : +- CometBroadcastExchange + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.web_sales + : : +- SubqueryBroadcast + : : +- BroadcastExchange + : : +- CometNativeColumnarToRow + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.date_dim + +- CometBroadcastExchange + +- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.customer + : +- CometBroadcastExchange + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.web_sales + : +- SubqueryBroadcast + : +- BroadcastExchange + : +- CometNativeColumnarToRow + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.date_dim + +- CometBroadcastExchange + +- CometFilter + +- CometNativeScan parquet spark_catalog.default.date_dim -Comet accelerated 28 out of 85 eligible operators (32%). Final plan contains 18 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 83 out of 91 eligible operators (91%). Final plan contains 5 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q12.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q12.native_datafusion/extended.txt index 15648408fc..a8c2531a38 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q12.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q12.native_datafusion/extended.txt @@ -3,33 +3,29 @@ TakeOrderedAndProject +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] +- CometNativeColumnarToRow +- CometSort - +- CometColumnarExchange - +- HashAggregate - +- CometNativeColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- Filter - : : +- ColumnarToRow - : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- CometNativeColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.item - +- BroadcastExchange - +- CometNativeColumnarToRow - +- CometProject - +- CometFilter - +- CometNativeScan parquet spark_catalog.default.date_dim + +- CometExchange + +- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.web_sales + : : +- SubqueryBroadcast + : : +- BroadcastExchange + : : +- CometNativeColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.item + +- CometBroadcastExchange + +- CometProject + +- CometFilter + +- CometNativeScan parquet spark_catalog.default.date_dim -Comet accelerated 12 out of 27 eligible operators (44%). Final plan contains 6 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 22 out of 27 eligible operators (81%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q14.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q14.native_datafusion/extended.txt index 7921f2fa61..50139381f2 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q14.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q14.native_datafusion/extended.txt @@ -1,404 +1,394 @@ -TakeOrderedAndProject -+- BroadcastHashJoin - :- Filter - : : +- Subquery - : : +- HashAggregate - : : +- CometNativeColumnarToRow - : : +- CometColumnarExchange - : : +- HashAggregate - : : +- Union - : : :- Project - : : : +- BroadcastHashJoin - : : : :- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- ReusedSubquery - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : :- Project - : : : +- BroadcastHashJoin - : : : :- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- ReusedSubquery - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- Project - : : +- BroadcastHashJoin - : : :- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : +- ReusedSubquery - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- HashAggregate - : +- CometNativeColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- BroadcastHashJoin - : : : :- Filter - : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : : +- Subquery - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- Project - : : : +- BroadcastHashJoin - : : : :- CometNativeColumnarToRow - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : +- BroadcastExchange - : : : +- BroadcastHashJoin - : : : :- CometNativeColumnarToRow - : : : : +- CometHashAggregate - : : : : +- CometColumnarExchange - : : : : +- HashAggregate - : : : : +- Project - : : : : +- BroadcastHashJoin - : : : : :- Project - : : : : : +- BroadcastHashJoin - : : : : : :- Filter - : : : : : : +- ColumnarToRow - : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometNativeColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : : +- BroadcastExchange - : : : : : +- BroadcastHashJoin - : : : : : :- CometNativeColumnarToRow - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : : +- BroadcastExchange - : : : : : +- Project - : : : : : +- BroadcastHashJoin - : : : : : :- Project - : : : : : : +- BroadcastHashJoin - : : : : : : :- Filter - : : : : : : : +- ColumnarToRow - : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : : : +- ReusedSubquery - : : : : : : +- BroadcastExchange - : : : : : : +- CometNativeColumnarToRow - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : : +- BroadcastExchange - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- Project - : : : +- BroadcastHashJoin - : : : :- Project - : : : : +- BroadcastHashJoin - : : : : :- Filter - : : : : : +- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : +- ReusedSubquery - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- BroadcastHashJoin - : : :- CometNativeColumnarToRow - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.item - : : +- BroadcastExchange - : : +- Project - : : +- BroadcastHashJoin - : : :- CometNativeColumnarToRow - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.item - : : +- BroadcastExchange - : : +- BroadcastHashJoin - : : :- CometNativeColumnarToRow - : : : +- CometHashAggregate - : : : +- CometColumnarExchange - : : : +- HashAggregate - : : : +- Project - : : : +- BroadcastHashJoin - : : : :- Project - : : : : +- BroadcastHashJoin - : : : : :- Filter - : : : : : +- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : +- BroadcastExchange - : : : : +- BroadcastHashJoin - : : : : :- CometNativeColumnarToRow - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : +- BroadcastExchange - : : : : +- Project - : : : : +- BroadcastHashJoin - : : : : :- Project - : : : : : +- BroadcastHashJoin - : : : : : :- Filter - : : : : : : +- ColumnarToRow - : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : : +- ReusedSubquery - : : : : : +- BroadcastExchange - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- Filter - : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- ReusedSubquery - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.item - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- CometNativeColumnarToRow - : +- CometProject - : +- CometFilter - : : +- Subquery - : : +- CometNativeColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- CometNativeScan parquet spark_catalog.default.date_dim - +- BroadcastExchange - +- Filter - : +- ReusedSubquery - +- HashAggregate - +- CometNativeColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- BroadcastHashJoin - : : :- Filter - : : : +- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : : +- Subquery - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- Project - : : +- BroadcastHashJoin - : : :- CometNativeColumnarToRow - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.item - : : +- BroadcastExchange - : : +- BroadcastHashJoin - : : :- CometNativeColumnarToRow - : : : +- CometHashAggregate - : : : +- CometColumnarExchange - : : : +- HashAggregate - : : : +- Project - : : : +- BroadcastHashJoin - : : : :- Project - : : : : +- BroadcastHashJoin - : : : : :- Filter - : : : : : +- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : +- BroadcastExchange - : : : : +- BroadcastHashJoin - : : : : :- CometNativeColumnarToRow - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : +- BroadcastExchange - : : : : +- Project - : : : : +- BroadcastHashJoin - : : : : :- Project - : : : : : +- BroadcastHashJoin - : : : : : :- Filter - : : : : : : +- ColumnarToRow - : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : : +- ReusedSubquery - : : : : : +- BroadcastExchange - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- Filter - : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- ReusedSubquery - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.item - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- BroadcastHashJoin - : :- CometNativeColumnarToRow - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.item - : +- BroadcastExchange - : +- Project - : +- BroadcastHashJoin - : :- CometNativeColumnarToRow - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.item - : +- BroadcastExchange - : +- BroadcastHashJoin - : :- CometNativeColumnarToRow - : : +- CometHashAggregate - : : +- CometColumnarExchange - : : +- HashAggregate - : : +- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- Filter - : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- BroadcastHashJoin - : : : :- CometNativeColumnarToRow - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : +- BroadcastExchange - : : : +- Project - : : : +- BroadcastHashJoin - : : : :- Project - : : : : +- BroadcastHashJoin - : : : : :- Filter - : : : : : +- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : +- ReusedSubquery - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Filter - : : : +- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : +- ReusedSubquery - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.item - : +- BroadcastExchange - : +- CometNativeColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.date_dim - +- BroadcastExchange - +- CometNativeColumnarToRow - +- CometProject - +- CometFilter - : +- Subquery - : +- CometNativeColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.date_dim - +- CometNativeScan parquet spark_catalog.default.date_dim +CometNativeColumnarToRow ++- CometTakeOrderedAndProject + +- CometBroadcastHashJoin + :- CometFilter + : : +- Subquery + : : +- CometNativeColumnarToRow + : : +- CometHashAggregate + : : +- CometExchange + : : +- CometHashAggregate + : : +- CometUnion + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometNativeScan parquet spark_catalog.default.store_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometNativeColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometNativeColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometNativeScan parquet spark_catalog.default.web_sales + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometNativeColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometNativeColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : : +- Subquery + : : : : : +- CometNativeColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : +- CometBroadcastExchange + : : : +- CometBroadcastHashJoin + : : : :- CometHashAggregate + : : : : +- CometExchange + : : : : +- CometHashAggregate + : : : : +- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometProject + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : : : +- SubqueryBroadcast + : : : : : : +- BroadcastExchange + : : : : : : +- CometNativeColumnarToRow + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- CometBroadcastExchange + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : : : +- CometBroadcastExchange + : : : : : +- CometProject + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometProject + : : : : : : +- CometBroadcastHashJoin + : : : : : : :- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : : : : +- SubqueryBroadcast + : : : : : : : +- BroadcastExchange + : : : : : : : +- CometNativeColumnarToRow + : : : : : : : +- CometProject + : : : : : : : +- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : +- CometBroadcastExchange + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : : : +- CometBroadcastExchange + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.web_sales + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometNativeColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.item + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.item + : : +- CometBroadcastExchange + : : +- CometBroadcastHashJoin + : : :- CometHashAggregate + : : : +- CometExchange + : : : +- CometHashAggregate + : : : +- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometNativeColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometBroadcastHashJoin + : : : : :- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometProject + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : : : +- SubqueryBroadcast + : : : : : : +- BroadcastExchange + : : : : : : +- CometNativeColumnarToRow + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- CometBroadcastExchange + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.web_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometNativeColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.item + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : : +- ReusedSubquery + : +- CometNativeScan parquet spark_catalog.default.date_dim + +- CometBroadcastExchange + +- CometFilter + : +- ReusedSubquery + +- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometNativeColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : : +- Subquery + : : : : +- CometNativeColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.item + : : +- CometBroadcastExchange + : : +- CometBroadcastHashJoin + : : :- CometHashAggregate + : : : +- CometExchange + : : : +- CometHashAggregate + : : : +- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometNativeColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometBroadcastHashJoin + : : : : :- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometProject + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : : : +- SubqueryBroadcast + : : : : : : +- BroadcastExchange + : : : : : : +- CometNativeColumnarToRow + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- CometBroadcastExchange + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.web_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometNativeColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.item + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometBroadcastHashJoin + : :- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.item + : +- CometBroadcastExchange + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.item + : +- CometBroadcastExchange + : +- CometBroadcastHashJoin + : :- CometHashAggregate + : : +- CometExchange + : : +- CometHashAggregate + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometNativeColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometNativeColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.web_sales + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometNativeColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.item + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.date_dim + +- CometBroadcastExchange + +- CometProject + +- CometFilter + : +- ReusedSubquery + +- CometNativeScan parquet spark_catalog.default.date_dim -Comet accelerated 128 out of 333 eligible operators (38%). Final plan contains 69 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 331 out of 371 eligible operators (89%). Final plan contains 21 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q14a.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q14a.native_datafusion/extended.txt index 0bc07e401e..636e5ce377 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q14a.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q14a.native_datafusion/extended.txt @@ -1,2779 +1,2750 @@ CometNativeColumnarToRow +- CometTakeOrderedAndProject +- CometHashAggregate - +- CometColumnarExchange - +- HashAggregate - +- Union - :- HashAggregate - : +- CometNativeColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Union - : :- Filter - : : : +- Subquery - : : : +- HashAggregate - : : : +- CometNativeColumnarToRow - : : : +- CometColumnarExchange - : : : +- HashAggregate - : : : +- Union - : : : :- Project - : : : : +- BroadcastHashJoin - : : : : :- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : +- ReusedSubquery - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : :- Project - : : : : +- BroadcastHashJoin - : : : : :- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- Project - : : : +- BroadcastHashJoin - : : : :- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- ReusedSubquery - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- HashAggregate - : : +- CometNativeColumnarToRow - : : +- CometColumnarExchange - : : +- HashAggregate - : : +- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- BroadcastHashJoin - : : : : :- Filter - : : : : : +- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : +- BroadcastExchange - : : : : +- Project - : : : : +- BroadcastHashJoin - : : : : :- CometNativeColumnarToRow - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : +- BroadcastExchange - : : : : +- BroadcastHashJoin - : : : : :- CometNativeColumnarToRow - : : : : : +- CometHashAggregate - : : : : : +- CometColumnarExchange - : : : : : +- HashAggregate - : : : : : +- Project - : : : : : +- BroadcastHashJoin - : : : : : :- Project - : : : : : : +- BroadcastHashJoin - : : : : : : :- Filter - : : : : : : : +- ColumnarToRow - : : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : : : +- SubqueryBroadcast - : : : : : : : +- BroadcastExchange - : : : : : : : +- CometNativeColumnarToRow - : : : : : : : +- CometProject - : : : : : : : +- CometFilter - : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : : : +- BroadcastExchange - : : : : : : +- BroadcastHashJoin - : : : : : : :- CometNativeColumnarToRow - : : : : : : : +- CometFilter - : : : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : : : +- BroadcastExchange - : : : : : : +- Project - : : : : : : +- BroadcastHashJoin - : : : : : : :- Project - : : : : : : : +- BroadcastHashJoin - : : : : : : : :- Filter - : : : : : : : : +- ColumnarToRow - : : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : : : : +- ReusedSubquery - : : : : : : : +- BroadcastExchange - : : : : : : : +- CometNativeColumnarToRow - : : : : : : : +- CometFilter - : : : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : : : +- BroadcastExchange - : : : : : : +- CometNativeColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : : +- BroadcastExchange - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : +- BroadcastExchange - : : : : +- Project - : : : : +- BroadcastHashJoin - : : : : :- Project - : : : : : +- BroadcastHashJoin - : : : : : :- Filter - : : : : : : +- ColumnarToRow - : : : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : : +- ReusedSubquery - : : : : : +- BroadcastExchange - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- BroadcastHashJoin - : : : :- CometNativeColumnarToRow - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : +- BroadcastExchange - : : : +- Project - : : : +- BroadcastHashJoin - : : : :- CometNativeColumnarToRow - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : +- BroadcastExchange - : : : +- BroadcastHashJoin - : : : :- CometNativeColumnarToRow - : : : : +- CometHashAggregate - : : : : +- CometColumnarExchange - : : : : +- HashAggregate - : : : : +- Project - : : : : +- BroadcastHashJoin - : : : : :- Project - : : : : : +- BroadcastHashJoin - : : : : : :- Filter - : : : : : : +- ColumnarToRow - : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometNativeColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : : +- BroadcastExchange - : : : : : +- BroadcastHashJoin - : : : : : :- CometNativeColumnarToRow - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : : +- BroadcastExchange - : : : : : +- Project - : : : : : +- BroadcastHashJoin - : : : : : :- Project - : : : : : : +- BroadcastHashJoin - : : : : : : :- Filter - : : : : : : : +- ColumnarToRow - : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : : : +- ReusedSubquery - : : : : : : +- BroadcastExchange - : : : : : : +- CometNativeColumnarToRow - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : : +- BroadcastExchange - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- Project - : : : +- BroadcastHashJoin - : : : :- Project - : : : : +- BroadcastHashJoin - : : : : :- Filter - : : : : : +- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : +- ReusedSubquery - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : :- Filter - : : : +- ReusedSubquery - : : +- HashAggregate - : : +- CometNativeColumnarToRow - : : +- CometColumnarExchange - : : +- HashAggregate - : : +- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- BroadcastHashJoin - : : : : :- Filter - : : : : : +- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : +- ReusedSubquery - : : : : +- BroadcastExchange - : : : : +- Project - : : : : +- BroadcastHashJoin - : : : : :- CometNativeColumnarToRow - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : +- BroadcastExchange - : : : : +- BroadcastHashJoin - : : : : :- CometNativeColumnarToRow - : : : : : +- CometHashAggregate - : : : : : +- CometColumnarExchange - : : : : : +- HashAggregate - : : : : : +- Project - : : : : : +- BroadcastHashJoin - : : : : : :- Project - : : : : : : +- BroadcastHashJoin - : : : : : : :- Filter - : : : : : : : +- ColumnarToRow - : : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : : : +- SubqueryBroadcast - : : : : : : : +- BroadcastExchange - : : : : : : : +- CometNativeColumnarToRow - : : : : : : : +- CometProject - : : : : : : : +- CometFilter - : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : : : +- BroadcastExchange - : : : : : : +- BroadcastHashJoin - : : : : : : :- CometNativeColumnarToRow - : : : : : : : +- CometFilter - : : : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : : : +- BroadcastExchange - : : : : : : +- Project - : : : : : : +- BroadcastHashJoin - : : : : : : :- Project - : : : : : : : +- BroadcastHashJoin - : : : : : : : :- Filter - : : : : : : : : +- ColumnarToRow - : : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : : : : +- ReusedSubquery - : : : : : : : +- BroadcastExchange - : : : : : : : +- CometNativeColumnarToRow - : : : : : : : +- CometFilter - : : : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : : : +- BroadcastExchange - : : : : : : +- CometNativeColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : : +- BroadcastExchange - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : +- BroadcastExchange - : : : : +- Project - : : : : +- BroadcastHashJoin - : : : : :- Project - : : : : : +- BroadcastHashJoin - : : : : : :- Filter - : : : : : : +- ColumnarToRow - : : : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : : +- ReusedSubquery - : : : : : +- BroadcastExchange - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- BroadcastHashJoin - : : : :- CometNativeColumnarToRow - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : +- BroadcastExchange - : : : +- Project - : : : +- BroadcastHashJoin - : : : :- CometNativeColumnarToRow - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : +- BroadcastExchange - : : : +- BroadcastHashJoin - : : : :- CometNativeColumnarToRow - : : : : +- CometHashAggregate - : : : : +- CometColumnarExchange - : : : : +- HashAggregate - : : : : +- Project - : : : : +- BroadcastHashJoin - : : : : :- Project - : : : : : +- BroadcastHashJoin - : : : : : :- Filter - : : : : : : +- ColumnarToRow - : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometNativeColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : : +- BroadcastExchange - : : : : : +- BroadcastHashJoin - : : : : : :- CometNativeColumnarToRow - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : : +- BroadcastExchange - : : : : : +- Project - : : : : : +- BroadcastHashJoin - : : : : : :- Project - : : : : : : +- BroadcastHashJoin - : : : : : : :- Filter - : : : : : : : +- ColumnarToRow - : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : : : +- ReusedSubquery - : : : : : : +- BroadcastExchange - : : : : : : +- CometNativeColumnarToRow - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : : +- BroadcastExchange - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- Project - : : : +- BroadcastHashJoin - : : : :- Project - : : : : +- BroadcastHashJoin - : : : : :- Filter - : : : : : +- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : +- ReusedSubquery - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- Filter - : : +- ReusedSubquery - : +- HashAggregate - : +- CometNativeColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- BroadcastHashJoin - : : : :- Filter - : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- ReusedSubquery - : : : +- BroadcastExchange - : : : +- Project - : : : +- BroadcastHashJoin - : : : :- CometNativeColumnarToRow - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : +- BroadcastExchange - : : : +- BroadcastHashJoin - : : : :- CometNativeColumnarToRow - : : : : +- CometHashAggregate - : : : : +- CometColumnarExchange - : : : : +- HashAggregate - : : : : +- Project - : : : : +- BroadcastHashJoin - : : : : :- Project - : : : : : +- BroadcastHashJoin - : : : : : :- Filter - : : : : : : +- ColumnarToRow - : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometNativeColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : : +- BroadcastExchange - : : : : : +- BroadcastHashJoin - : : : : : :- CometNativeColumnarToRow - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : : +- BroadcastExchange - : : : : : +- Project - : : : : : +- BroadcastHashJoin - : : : : : :- Project - : : : : : : +- BroadcastHashJoin - : : : : : : :- Filter - : : : : : : : +- ColumnarToRow - : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : : : +- ReusedSubquery - : : : : : : +- BroadcastExchange - : : : : : : +- CometNativeColumnarToRow - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : : +- BroadcastExchange - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- Project - : : : +- BroadcastHashJoin - : : : :- Project - : : : : +- BroadcastHashJoin - : : : : :- Filter - : : : : : +- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : +- ReusedSubquery - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- BroadcastHashJoin - : : :- CometNativeColumnarToRow - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.item - : : +- BroadcastExchange - : : +- Project - : : +- BroadcastHashJoin - : : :- CometNativeColumnarToRow - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.item - : : +- BroadcastExchange - : : +- BroadcastHashJoin - : : :- CometNativeColumnarToRow - : : : +- CometHashAggregate - : : : +- CometColumnarExchange - : : : +- HashAggregate - : : : +- Project - : : : +- BroadcastHashJoin - : : : :- Project - : : : : +- BroadcastHashJoin - : : : : :- Filter - : : : : : +- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : +- BroadcastExchange - : : : : +- BroadcastHashJoin - : : : : :- CometNativeColumnarToRow - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : +- BroadcastExchange - : : : : +- Project - : : : : +- BroadcastHashJoin - : : : : :- Project - : : : : : +- BroadcastHashJoin - : : : : : :- Filter - : : : : : : +- ColumnarToRow - : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : : +- ReusedSubquery - : : : : : +- BroadcastExchange - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- Filter - : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- ReusedSubquery - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.item - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- CometNativeColumnarToRow + +- CometExchange + +- CometHashAggregate + +- CometUnion + :- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometUnion + : :- CometFilter + : : : +- Subquery + : : : +- CometNativeColumnarToRow + : : : +- CometHashAggregate + : : : +- CometExchange + : : : +- CometHashAggregate + : : : +- CometUnion + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometNativeScan parquet spark_catalog.default.store_sales + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometNativeColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometNativeColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometNativeScan parquet spark_catalog.default.web_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometNativeColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometHashAggregate + : : +- CometExchange + : : +- CometHashAggregate + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometBroadcastHashJoin + : : : : :- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometNativeColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : : +- CometBroadcastExchange + : : : : +- CometBroadcastHashJoin + : : : : :- CometHashAggregate + : : : : : +- CometExchange + : : : : : +- CometHashAggregate + : : : : : +- CometProject + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometProject + : : : : : : +- CometBroadcastHashJoin + : : : : : : :- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : : : : +- SubqueryBroadcast + : : : : : : : +- BroadcastExchange + : : : : : : : +- CometNativeColumnarToRow + : : : : : : : +- CometProject + : : : : : : : +- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : +- CometBroadcastExchange + : : : : : : +- CometBroadcastHashJoin + : : : : : : :- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : : : : +- CometBroadcastExchange + : : : : : : +- CometProject + : : : : : : +- CometBroadcastHashJoin + : : : : : : :- CometProject + : : : : : : : +- CometBroadcastHashJoin + : : : : : : : :- CometFilter + : : : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : : : : : +- SubqueryBroadcast + : : : : : : : : +- BroadcastExchange + : : : : : : : : +- CometNativeColumnarToRow + : : : : : : : : +- CometProject + : : : : : : : : +- CometFilter + : : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : : +- CometBroadcastExchange + : : : : : : : +- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : : : : +- CometBroadcastExchange + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- CometBroadcastExchange + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometProject + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.web_sales + : : : : : : +- SubqueryBroadcast + : : : : : : +- BroadcastExchange + : : : : : : +- CometNativeColumnarToRow + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- CometBroadcastExchange + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : +- CometBroadcastExchange + : : : +- CometBroadcastHashJoin + : : : :- CometHashAggregate + : : : : +- CometExchange + : : : : +- CometHashAggregate + : : : : +- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometProject + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : : : +- SubqueryBroadcast + : : : : : : +- BroadcastExchange + : : : : : : +- CometNativeColumnarToRow + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- CometBroadcastExchange + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : : : +- CometBroadcastExchange + : : : : : +- CometProject + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometProject + : : : : : : +- CometBroadcastHashJoin + : : : : : : :- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : : : : +- SubqueryBroadcast + : : : : : : : +- BroadcastExchange + : : : : : : : +- CometNativeColumnarToRow + : : : : : : : +- CometProject + : : : : : : : +- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : +- CometBroadcastExchange + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : : : +- CometBroadcastExchange + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.web_sales + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometNativeColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : :- CometFilter + : : : +- ReusedSubquery + : : +- CometHashAggregate + : : +- CometExchange + : : +- CometHashAggregate + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometBroadcastHashJoin + : : : : :- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometNativeColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : : +- CometBroadcastExchange + : : : : +- CometBroadcastHashJoin + : : : : :- CometHashAggregate + : : : : : +- CometExchange + : : : : : +- CometHashAggregate + : : : : : +- CometProject + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometProject + : : : : : : +- CometBroadcastHashJoin + : : : : : : :- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : : : : +- SubqueryBroadcast + : : : : : : : +- BroadcastExchange + : : : : : : : +- CometNativeColumnarToRow + : : : : : : : +- CometProject + : : : : : : : +- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : +- CometBroadcastExchange + : : : : : : +- CometBroadcastHashJoin + : : : : : : :- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : : : : +- CometBroadcastExchange + : : : : : : +- CometProject + : : : : : : +- CometBroadcastHashJoin + : : : : : : :- CometProject + : : : : : : : +- CometBroadcastHashJoin + : : : : : : : :- CometFilter + : : : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : : : : : +- SubqueryBroadcast + : : : : : : : : +- BroadcastExchange + : : : : : : : : +- CometNativeColumnarToRow + : : : : : : : : +- CometProject + : : : : : : : : +- CometFilter + : : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : : +- CometBroadcastExchange + : : : : : : : +- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : : : : +- CometBroadcastExchange + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- CometBroadcastExchange + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometProject + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.web_sales + : : : : : : +- SubqueryBroadcast + : : : : : : +- BroadcastExchange + : : : : : : +- CometNativeColumnarToRow + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- CometBroadcastExchange + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : +- CometBroadcastExchange + : : : +- CometBroadcastHashJoin + : : : :- CometHashAggregate + : : : : +- CometExchange + : : : : +- CometHashAggregate + : : : : +- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometProject + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : : : +- SubqueryBroadcast + : : : : : : +- BroadcastExchange + : : : : : : +- CometNativeColumnarToRow + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- CometBroadcastExchange + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : : : +- CometBroadcastExchange + : : : : : +- CometProject + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometProject + : : : : : : +- CometBroadcastHashJoin + : : : : : : :- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : : : : +- SubqueryBroadcast + : : : : : : : +- BroadcastExchange + : : : : : : : +- CometNativeColumnarToRow + : : : : : : : +- CometProject + : : : : : : : +- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : +- CometBroadcastExchange + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : : : +- CometBroadcastExchange + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.web_sales + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometNativeColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometFilter + : : +- ReusedSubquery + : +- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.web_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometNativeColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : +- CometBroadcastExchange + : : : +- CometBroadcastHashJoin + : : : :- CometHashAggregate + : : : : +- CometExchange + : : : : +- CometHashAggregate + : : : : +- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometProject + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : : : +- SubqueryBroadcast + : : : : : : +- BroadcastExchange + : : : : : : +- CometNativeColumnarToRow + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- CometBroadcastExchange + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : : : +- CometBroadcastExchange + : : : : : +- CometProject + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometProject + : : : : : : +- CometBroadcastHashJoin + : : : : : : :- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : : : : +- SubqueryBroadcast + : : : : : : : +- BroadcastExchange + : : : : : : : +- CometNativeColumnarToRow + : : : : : : : +- CometProject + : : : : : : : +- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : +- CometBroadcastExchange + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : : : +- CometBroadcastExchange + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.web_sales + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometNativeColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.item + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.item + : : +- CometBroadcastExchange + : : +- CometBroadcastHashJoin + : : :- CometHashAggregate + : : : +- CometExchange + : : : +- CometHashAggregate + : : : +- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometNativeColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometBroadcastHashJoin + : : : : :- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometProject + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : : : +- SubqueryBroadcast + : : : : : : +- BroadcastExchange + : : : : : : +- CometNativeColumnarToRow + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- CometBroadcastExchange + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.web_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometNativeColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.item + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.date_dim + :- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometUnion + : :- CometFilter + : : : +- Subquery + : : : +- CometNativeColumnarToRow + : : : +- CometHashAggregate + : : : +- CometExchange + : : : +- CometHashAggregate + : : : +- CometUnion + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometNativeScan parquet spark_catalog.default.store_sales + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometNativeColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometNativeColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometNativeScan parquet spark_catalog.default.web_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometNativeColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometHashAggregate + : : +- CometExchange + : : +- CometHashAggregate + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometBroadcastHashJoin + : : : : :- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometNativeColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : : +- CometBroadcastExchange + : : : : +- CometBroadcastHashJoin + : : : : :- CometHashAggregate + : : : : : +- CometExchange + : : : : : +- CometHashAggregate + : : : : : +- CometProject + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometProject + : : : : : : +- CometBroadcastHashJoin + : : : : : : :- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : : : : +- SubqueryBroadcast + : : : : : : : +- BroadcastExchange + : : : : : : : +- CometNativeColumnarToRow + : : : : : : : +- CometProject + : : : : : : : +- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : +- CometBroadcastExchange + : : : : : : +- CometBroadcastHashJoin + : : : : : : :- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : : : : +- CometBroadcastExchange + : : : : : : +- CometProject + : : : : : : +- CometBroadcastHashJoin + : : : : : : :- CometProject + : : : : : : : +- CometBroadcastHashJoin + : : : : : : : :- CometFilter + : : : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : : : : : +- SubqueryBroadcast + : : : : : : : : +- BroadcastExchange + : : : : : : : : +- CometNativeColumnarToRow + : : : : : : : : +- CometProject + : : : : : : : : +- CometFilter + : : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : : +- CometBroadcastExchange + : : : : : : : +- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : : : : +- CometBroadcastExchange + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- CometBroadcastExchange + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometProject + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.web_sales + : : : : : : +- SubqueryBroadcast + : : : : : : +- BroadcastExchange + : : : : : : +- CometNativeColumnarToRow + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- CometBroadcastExchange + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : +- CometBroadcastExchange + : : : +- CometBroadcastHashJoin + : : : :- CometHashAggregate + : : : : +- CometExchange + : : : : +- CometHashAggregate + : : : : +- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometProject + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : : : +- SubqueryBroadcast + : : : : : : +- BroadcastExchange + : : : : : : +- CometNativeColumnarToRow + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- CometBroadcastExchange + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : : : +- CometBroadcastExchange + : : : : : +- CometProject + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometProject + : : : : : : +- CometBroadcastHashJoin + : : : : : : :- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : : : : +- SubqueryBroadcast + : : : : : : : +- BroadcastExchange + : : : : : : : +- CometNativeColumnarToRow + : : : : : : : +- CometProject + : : : : : : : +- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : +- CometBroadcastExchange + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : : : +- CometBroadcastExchange + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.web_sales + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometNativeColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : :- CometFilter + : : : +- ReusedSubquery + : : +- CometHashAggregate + : : +- CometExchange + : : +- CometHashAggregate + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometBroadcastHashJoin + : : : : :- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometNativeColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : : +- CometBroadcastExchange + : : : : +- CometBroadcastHashJoin + : : : : :- CometHashAggregate + : : : : : +- CometExchange + : : : : : +- CometHashAggregate + : : : : : +- CometProject + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometProject + : : : : : : +- CometBroadcastHashJoin + : : : : : : :- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : : : : +- SubqueryBroadcast + : : : : : : : +- BroadcastExchange + : : : : : : : +- CometNativeColumnarToRow + : : : : : : : +- CometProject + : : : : : : : +- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : +- CometBroadcastExchange + : : : : : : +- CometBroadcastHashJoin + : : : : : : :- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : : : : +- CometBroadcastExchange + : : : : : : +- CometProject + : : : : : : +- CometBroadcastHashJoin + : : : : : : :- CometProject + : : : : : : : +- CometBroadcastHashJoin + : : : : : : : :- CometFilter + : : : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : : : : : +- SubqueryBroadcast + : : : : : : : : +- BroadcastExchange + : : : : : : : : +- CometNativeColumnarToRow + : : : : : : : : +- CometProject + : : : : : : : : +- CometFilter + : : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : : +- CometBroadcastExchange + : : : : : : : +- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : : : : +- CometBroadcastExchange + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- CometBroadcastExchange + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometProject + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.web_sales + : : : : : : +- SubqueryBroadcast + : : : : : : +- BroadcastExchange + : : : : : : +- CometNativeColumnarToRow + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- CometBroadcastExchange + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : +- CometBroadcastExchange + : : : +- CometBroadcastHashJoin + : : : :- CometHashAggregate + : : : : +- CometExchange + : : : : +- CometHashAggregate + : : : : +- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometProject + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : : : +- SubqueryBroadcast + : : : : : : +- BroadcastExchange + : : : : : : +- CometNativeColumnarToRow + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- CometBroadcastExchange + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : : : +- CometBroadcastExchange + : : : : : +- CometProject + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometProject + : : : : : : +- CometBroadcastHashJoin + : : : : : : :- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : : : : +- SubqueryBroadcast + : : : : : : : +- BroadcastExchange + : : : : : : : +- CometNativeColumnarToRow + : : : : : : : +- CometProject + : : : : : : : +- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : +- CometBroadcastExchange + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : : : +- CometBroadcastExchange + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.web_sales + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometNativeColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometFilter + : : +- ReusedSubquery + : +- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.web_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometNativeColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : +- CometBroadcastExchange + : : : +- CometBroadcastHashJoin + : : : :- CometHashAggregate + : : : : +- CometExchange + : : : : +- CometHashAggregate + : : : : +- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometProject + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : : : +- SubqueryBroadcast + : : : : : : +- BroadcastExchange + : : : : : : +- CometNativeColumnarToRow + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- CometBroadcastExchange + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : : : +- CometBroadcastExchange + : : : : : +- CometProject + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometProject + : : : : : : +- CometBroadcastHashJoin + : : : : : : :- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : : : : +- SubqueryBroadcast + : : : : : : : +- BroadcastExchange + : : : : : : : +- CometNativeColumnarToRow + : : : : : : : +- CometProject + : : : : : : : +- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : +- CometBroadcastExchange + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : : : +- CometBroadcastExchange + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.web_sales + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometNativeColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.item + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.item + : : +- CometBroadcastExchange + : : +- CometBroadcastHashJoin + : : :- CometHashAggregate + : : : +- CometExchange + : : : +- CometHashAggregate + : : : +- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometNativeColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometBroadcastHashJoin + : : : : :- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometProject + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : : : +- SubqueryBroadcast + : : : : : : +- BroadcastExchange + : : : : : : +- CometNativeColumnarToRow + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- CometBroadcastExchange + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.web_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometNativeColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.item + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange : +- CometProject : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.date_dim - :- HashAggregate - : +- CometNativeColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- HashAggregate - : +- CometNativeColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Union - : :- Filter - : : : +- Subquery - : : : +- HashAggregate - : : : +- CometNativeColumnarToRow - : : : +- CometColumnarExchange - : : : +- HashAggregate - : : : +- Union - : : : :- Project - : : : : +- BroadcastHashJoin - : : : : :- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : +- ReusedSubquery - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : :- Project - : : : : +- BroadcastHashJoin - : : : : :- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- Project - : : : +- BroadcastHashJoin - : : : :- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- ReusedSubquery - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- HashAggregate - : : +- CometNativeColumnarToRow - : : +- CometColumnarExchange - : : +- HashAggregate - : : +- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- BroadcastHashJoin - : : : : :- Filter - : : : : : +- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : +- BroadcastExchange - : : : : +- Project - : : : : +- BroadcastHashJoin - : : : : :- CometNativeColumnarToRow - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : +- BroadcastExchange - : : : : +- BroadcastHashJoin - : : : : :- CometNativeColumnarToRow - : : : : : +- CometHashAggregate - : : : : : +- CometColumnarExchange - : : : : : +- HashAggregate - : : : : : +- Project - : : : : : +- BroadcastHashJoin - : : : : : :- Project - : : : : : : +- BroadcastHashJoin - : : : : : : :- Filter - : : : : : : : +- ColumnarToRow - : : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : : : +- SubqueryBroadcast - : : : : : : : +- BroadcastExchange - : : : : : : : +- CometNativeColumnarToRow - : : : : : : : +- CometProject - : : : : : : : +- CometFilter - : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : : : +- BroadcastExchange - : : : : : : +- BroadcastHashJoin - : : : : : : :- CometNativeColumnarToRow - : : : : : : : +- CometFilter - : : : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : : : +- BroadcastExchange - : : : : : : +- Project - : : : : : : +- BroadcastHashJoin - : : : : : : :- Project - : : : : : : : +- BroadcastHashJoin - : : : : : : : :- Filter - : : : : : : : : +- ColumnarToRow - : : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : : : : +- ReusedSubquery - : : : : : : : +- BroadcastExchange - : : : : : : : +- CometNativeColumnarToRow - : : : : : : : +- CometFilter - : : : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : : : +- BroadcastExchange - : : : : : : +- CometNativeColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : : +- BroadcastExchange - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : +- BroadcastExchange - : : : : +- Project - : : : : +- BroadcastHashJoin - : : : : :- Project - : : : : : +- BroadcastHashJoin - : : : : : :- Filter - : : : : : : +- ColumnarToRow - : : : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : : +- ReusedSubquery - : : : : : +- BroadcastExchange - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- BroadcastHashJoin - : : : :- CometNativeColumnarToRow - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : +- BroadcastExchange - : : : +- Project - : : : +- BroadcastHashJoin - : : : :- CometNativeColumnarToRow - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : +- BroadcastExchange - : : : +- BroadcastHashJoin - : : : :- CometNativeColumnarToRow - : : : : +- CometHashAggregate - : : : : +- CometColumnarExchange - : : : : +- HashAggregate - : : : : +- Project - : : : : +- BroadcastHashJoin - : : : : :- Project - : : : : : +- BroadcastHashJoin - : : : : : :- Filter - : : : : : : +- ColumnarToRow - : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometNativeColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : : +- BroadcastExchange - : : : : : +- BroadcastHashJoin - : : : : : :- CometNativeColumnarToRow - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : : +- BroadcastExchange - : : : : : +- Project - : : : : : +- BroadcastHashJoin - : : : : : :- Project - : : : : : : +- BroadcastHashJoin - : : : : : : :- Filter - : : : : : : : +- ColumnarToRow - : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : : : +- ReusedSubquery - : : : : : : +- BroadcastExchange - : : : : : : +- CometNativeColumnarToRow - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : : +- BroadcastExchange - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- Project - : : : +- BroadcastHashJoin - : : : :- Project - : : : : +- BroadcastHashJoin - : : : : :- Filter - : : : : : +- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : +- ReusedSubquery - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : :- Filter - : : : +- ReusedSubquery - : : +- HashAggregate - : : +- CometNativeColumnarToRow - : : +- CometColumnarExchange - : : +- HashAggregate - : : +- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- BroadcastHashJoin - : : : : :- Filter - : : : : : +- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : +- ReusedSubquery - : : : : +- BroadcastExchange - : : : : +- Project - : : : : +- BroadcastHashJoin - : : : : :- CometNativeColumnarToRow - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : +- BroadcastExchange - : : : : +- BroadcastHashJoin - : : : : :- CometNativeColumnarToRow - : : : : : +- CometHashAggregate - : : : : : +- CometColumnarExchange - : : : : : +- HashAggregate - : : : : : +- Project - : : : : : +- BroadcastHashJoin - : : : : : :- Project - : : : : : : +- BroadcastHashJoin - : : : : : : :- Filter - : : : : : : : +- ColumnarToRow - : : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : : : +- SubqueryBroadcast - : : : : : : : +- BroadcastExchange - : : : : : : : +- CometNativeColumnarToRow - : : : : : : : +- CometProject - : : : : : : : +- CometFilter - : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : : : +- BroadcastExchange - : : : : : : +- BroadcastHashJoin - : : : : : : :- CometNativeColumnarToRow - : : : : : : : +- CometFilter - : : : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : : : +- BroadcastExchange - : : : : : : +- Project - : : : : : : +- BroadcastHashJoin - : : : : : : :- Project - : : : : : : : +- BroadcastHashJoin - : : : : : : : :- Filter - : : : : : : : : +- ColumnarToRow - : : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : : : : +- ReusedSubquery - : : : : : : : +- BroadcastExchange - : : : : : : : +- CometNativeColumnarToRow - : : : : : : : +- CometFilter - : : : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : : : +- BroadcastExchange - : : : : : : +- CometNativeColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : : +- BroadcastExchange - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : +- BroadcastExchange - : : : : +- Project - : : : : +- BroadcastHashJoin - : : : : :- Project - : : : : : +- BroadcastHashJoin - : : : : : :- Filter - : : : : : : +- ColumnarToRow - : : : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : : +- ReusedSubquery - : : : : : +- BroadcastExchange - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- BroadcastHashJoin - : : : :- CometNativeColumnarToRow - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : +- BroadcastExchange - : : : +- Project - : : : +- BroadcastHashJoin - : : : :- CometNativeColumnarToRow - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : +- BroadcastExchange - : : : +- BroadcastHashJoin - : : : :- CometNativeColumnarToRow - : : : : +- CometHashAggregate - : : : : +- CometColumnarExchange - : : : : +- HashAggregate - : : : : +- Project - : : : : +- BroadcastHashJoin - : : : : :- Project - : : : : : +- BroadcastHashJoin - : : : : : :- Filter - : : : : : : +- ColumnarToRow - : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometNativeColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : : +- BroadcastExchange - : : : : : +- BroadcastHashJoin - : : : : : :- CometNativeColumnarToRow - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : : +- BroadcastExchange - : : : : : +- Project - : : : : : +- BroadcastHashJoin - : : : : : :- Project - : : : : : : +- BroadcastHashJoin - : : : : : : :- Filter - : : : : : : : +- ColumnarToRow - : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : : : +- ReusedSubquery - : : : : : : +- BroadcastExchange - : : : : : : +- CometNativeColumnarToRow - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : : +- BroadcastExchange - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- Project - : : : +- BroadcastHashJoin - : : : :- Project - : : : : +- BroadcastHashJoin - : : : : :- Filter - : : : : : +- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : +- ReusedSubquery - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- Filter - : : +- ReusedSubquery - : +- HashAggregate - : +- CometNativeColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- BroadcastHashJoin - : : : :- Filter - : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- ReusedSubquery - : : : +- BroadcastExchange - : : : +- Project - : : : +- BroadcastHashJoin - : : : :- CometNativeColumnarToRow - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : +- BroadcastExchange - : : : +- BroadcastHashJoin - : : : :- CometNativeColumnarToRow - : : : : +- CometHashAggregate - : : : : +- CometColumnarExchange - : : : : +- HashAggregate - : : : : +- Project - : : : : +- BroadcastHashJoin - : : : : :- Project - : : : : : +- BroadcastHashJoin - : : : : : :- Filter - : : : : : : +- ColumnarToRow - : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometNativeColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : : +- BroadcastExchange - : : : : : +- BroadcastHashJoin - : : : : : :- CometNativeColumnarToRow - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : : +- BroadcastExchange - : : : : : +- Project - : : : : : +- BroadcastHashJoin - : : : : : :- Project - : : : : : : +- BroadcastHashJoin - : : : : : : :- Filter - : : : : : : : +- ColumnarToRow - : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : : : +- ReusedSubquery - : : : : : : +- BroadcastExchange - : : : : : : +- CometNativeColumnarToRow - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : : +- BroadcastExchange - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- Project - : : : +- BroadcastHashJoin - : : : :- Project - : : : : +- BroadcastHashJoin - : : : : :- Filter - : : : : : +- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : +- ReusedSubquery - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- BroadcastHashJoin - : : :- CometNativeColumnarToRow - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.item - : : +- BroadcastExchange - : : +- Project - : : +- BroadcastHashJoin - : : :- CometNativeColumnarToRow - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.item - : : +- BroadcastExchange - : : +- BroadcastHashJoin - : : :- CometNativeColumnarToRow - : : : +- CometHashAggregate - : : : +- CometColumnarExchange - : : : +- HashAggregate - : : : +- Project - : : : +- BroadcastHashJoin - : : : :- Project - : : : : +- BroadcastHashJoin - : : : : :- Filter - : : : : : +- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : +- BroadcastExchange - : : : : +- BroadcastHashJoin - : : : : :- CometNativeColumnarToRow - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : +- BroadcastExchange - : : : : +- Project - : : : : +- BroadcastHashJoin - : : : : :- Project - : : : : : +- BroadcastHashJoin - : : : : : :- Filter - : : : : : : +- ColumnarToRow - : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : : +- ReusedSubquery - : : : : : +- BroadcastExchange - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- Filter - : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- ReusedSubquery - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.item - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- CometNativeColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.date_dim - :- HashAggregate - : +- CometNativeColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- HashAggregate - : +- CometNativeColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Union - : :- Filter - : : : +- Subquery - : : : +- HashAggregate - : : : +- CometNativeColumnarToRow - : : : +- CometColumnarExchange - : : : +- HashAggregate - : : : +- Union - : : : :- Project - : : : : +- BroadcastHashJoin - : : : : :- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : +- ReusedSubquery - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : :- Project - : : : : +- BroadcastHashJoin - : : : : :- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- Project - : : : +- BroadcastHashJoin - : : : :- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- ReusedSubquery - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- HashAggregate - : : +- CometNativeColumnarToRow - : : +- CometColumnarExchange - : : +- HashAggregate - : : +- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- BroadcastHashJoin - : : : : :- Filter - : : : : : +- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : +- BroadcastExchange - : : : : +- Project - : : : : +- BroadcastHashJoin - : : : : :- CometNativeColumnarToRow - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : +- BroadcastExchange - : : : : +- BroadcastHashJoin - : : : : :- CometNativeColumnarToRow - : : : : : +- CometHashAggregate - : : : : : +- CometColumnarExchange - : : : : : +- HashAggregate - : : : : : +- Project - : : : : : +- BroadcastHashJoin - : : : : : :- Project - : : : : : : +- BroadcastHashJoin - : : : : : : :- Filter - : : : : : : : +- ColumnarToRow - : : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : : : +- SubqueryBroadcast - : : : : : : : +- BroadcastExchange - : : : : : : : +- CometNativeColumnarToRow - : : : : : : : +- CometProject - : : : : : : : +- CometFilter - : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : : : +- BroadcastExchange - : : : : : : +- BroadcastHashJoin - : : : : : : :- CometNativeColumnarToRow - : : : : : : : +- CometFilter - : : : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : : : +- BroadcastExchange - : : : : : : +- Project - : : : : : : +- BroadcastHashJoin - : : : : : : :- Project - : : : : : : : +- BroadcastHashJoin - : : : : : : : :- Filter - : : : : : : : : +- ColumnarToRow - : : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : : : : +- ReusedSubquery - : : : : : : : +- BroadcastExchange - : : : : : : : +- CometNativeColumnarToRow - : : : : : : : +- CometFilter - : : : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : : : +- BroadcastExchange - : : : : : : +- CometNativeColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : : +- BroadcastExchange - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : +- BroadcastExchange - : : : : +- Project - : : : : +- BroadcastHashJoin - : : : : :- Project - : : : : : +- BroadcastHashJoin - : : : : : :- Filter - : : : : : : +- ColumnarToRow - : : : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : : +- ReusedSubquery - : : : : : +- BroadcastExchange - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- BroadcastHashJoin - : : : :- CometNativeColumnarToRow - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : +- BroadcastExchange - : : : +- Project - : : : +- BroadcastHashJoin - : : : :- CometNativeColumnarToRow - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : +- BroadcastExchange - : : : +- BroadcastHashJoin - : : : :- CometNativeColumnarToRow - : : : : +- CometHashAggregate - : : : : +- CometColumnarExchange - : : : : +- HashAggregate - : : : : +- Project - : : : : +- BroadcastHashJoin - : : : : :- Project - : : : : : +- BroadcastHashJoin - : : : : : :- Filter - : : : : : : +- ColumnarToRow - : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometNativeColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : : +- BroadcastExchange - : : : : : +- BroadcastHashJoin - : : : : : :- CometNativeColumnarToRow - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : : +- BroadcastExchange - : : : : : +- Project - : : : : : +- BroadcastHashJoin - : : : : : :- Project - : : : : : : +- BroadcastHashJoin - : : : : : : :- Filter - : : : : : : : +- ColumnarToRow - : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : : : +- ReusedSubquery - : : : : : : +- BroadcastExchange - : : : : : : +- CometNativeColumnarToRow - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : : +- BroadcastExchange - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- Project - : : : +- BroadcastHashJoin - : : : :- Project - : : : : +- BroadcastHashJoin - : : : : :- Filter - : : : : : +- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : +- ReusedSubquery - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : :- Filter - : : : +- ReusedSubquery - : : +- HashAggregate - : : +- CometNativeColumnarToRow - : : +- CometColumnarExchange - : : +- HashAggregate - : : +- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- BroadcastHashJoin - : : : : :- Filter - : : : : : +- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : +- ReusedSubquery - : : : : +- BroadcastExchange - : : : : +- Project - : : : : +- BroadcastHashJoin - : : : : :- CometNativeColumnarToRow - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : +- BroadcastExchange - : : : : +- BroadcastHashJoin - : : : : :- CometNativeColumnarToRow - : : : : : +- CometHashAggregate - : : : : : +- CometColumnarExchange - : : : : : +- HashAggregate - : : : : : +- Project - : : : : : +- BroadcastHashJoin - : : : : : :- Project - : : : : : : +- BroadcastHashJoin - : : : : : : :- Filter - : : : : : : : +- ColumnarToRow - : : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : : : +- SubqueryBroadcast - : : : : : : : +- BroadcastExchange - : : : : : : : +- CometNativeColumnarToRow - : : : : : : : +- CometProject - : : : : : : : +- CometFilter - : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : : : +- BroadcastExchange - : : : : : : +- BroadcastHashJoin - : : : : : : :- CometNativeColumnarToRow - : : : : : : : +- CometFilter - : : : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : : : +- BroadcastExchange - : : : : : : +- Project - : : : : : : +- BroadcastHashJoin - : : : : : : :- Project - : : : : : : : +- BroadcastHashJoin - : : : : : : : :- Filter - : : : : : : : : +- ColumnarToRow - : : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : : : : +- ReusedSubquery - : : : : : : : +- BroadcastExchange - : : : : : : : +- CometNativeColumnarToRow - : : : : : : : +- CometFilter - : : : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : : : +- BroadcastExchange - : : : : : : +- CometNativeColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : : +- BroadcastExchange - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : +- BroadcastExchange - : : : : +- Project - : : : : +- BroadcastHashJoin - : : : : :- Project - : : : : : +- BroadcastHashJoin - : : : : : :- Filter - : : : : : : +- ColumnarToRow - : : : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : : +- ReusedSubquery - : : : : : +- BroadcastExchange - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- BroadcastHashJoin - : : : :- CometNativeColumnarToRow - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : +- BroadcastExchange - : : : +- Project - : : : +- BroadcastHashJoin - : : : :- CometNativeColumnarToRow - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : +- BroadcastExchange - : : : +- BroadcastHashJoin - : : : :- CometNativeColumnarToRow - : : : : +- CometHashAggregate - : : : : +- CometColumnarExchange - : : : : +- HashAggregate - : : : : +- Project - : : : : +- BroadcastHashJoin - : : : : :- Project - : : : : : +- BroadcastHashJoin - : : : : : :- Filter - : : : : : : +- ColumnarToRow - : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometNativeColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : : +- BroadcastExchange - : : : : : +- BroadcastHashJoin - : : : : : :- CometNativeColumnarToRow - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : : +- BroadcastExchange - : : : : : +- Project - : : : : : +- BroadcastHashJoin - : : : : : :- Project - : : : : : : +- BroadcastHashJoin - : : : : : : :- Filter - : : : : : : : +- ColumnarToRow - : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : : : +- ReusedSubquery - : : : : : : +- BroadcastExchange - : : : : : : +- CometNativeColumnarToRow - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : : +- BroadcastExchange - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- Project - : : : +- BroadcastHashJoin - : : : :- Project - : : : : +- BroadcastHashJoin - : : : : :- Filter - : : : : : +- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : +- ReusedSubquery - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- Filter - : : +- ReusedSubquery - : +- HashAggregate - : +- CometNativeColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- BroadcastHashJoin - : : : :- Filter - : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- ReusedSubquery - : : : +- BroadcastExchange - : : : +- Project - : : : +- BroadcastHashJoin - : : : :- CometNativeColumnarToRow - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : +- BroadcastExchange - : : : +- BroadcastHashJoin - : : : :- CometNativeColumnarToRow - : : : : +- CometHashAggregate - : : : : +- CometColumnarExchange - : : : : +- HashAggregate - : : : : +- Project - : : : : +- BroadcastHashJoin - : : : : :- Project - : : : : : +- BroadcastHashJoin - : : : : : :- Filter - : : : : : : +- ColumnarToRow - : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometNativeColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : : +- BroadcastExchange - : : : : : +- BroadcastHashJoin - : : : : : :- CometNativeColumnarToRow - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : : +- BroadcastExchange - : : : : : +- Project - : : : : : +- BroadcastHashJoin - : : : : : :- Project - : : : : : : +- BroadcastHashJoin - : : : : : : :- Filter - : : : : : : : +- ColumnarToRow - : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : : : +- ReusedSubquery - : : : : : : +- BroadcastExchange - : : : : : : +- CometNativeColumnarToRow - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : : +- BroadcastExchange - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- Project - : : : +- BroadcastHashJoin - : : : :- Project - : : : : +- BroadcastHashJoin - : : : : :- Filter - : : : : : +- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : +- ReusedSubquery - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- BroadcastHashJoin - : : :- CometNativeColumnarToRow - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.item - : : +- BroadcastExchange - : : +- Project - : : +- BroadcastHashJoin - : : :- CometNativeColumnarToRow - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.item - : : +- BroadcastExchange - : : +- BroadcastHashJoin - : : :- CometNativeColumnarToRow - : : : +- CometHashAggregate - : : : +- CometColumnarExchange - : : : +- HashAggregate - : : : +- Project - : : : +- BroadcastHashJoin - : : : :- Project - : : : : +- BroadcastHashJoin - : : : : :- Filter - : : : : : +- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : +- BroadcastExchange - : : : : +- BroadcastHashJoin - : : : : :- CometNativeColumnarToRow - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : +- BroadcastExchange - : : : : +- Project - : : : : +- BroadcastHashJoin - : : : : :- Project - : : : : : +- BroadcastHashJoin - : : : : : :- Filter - : : : : : : +- ColumnarToRow - : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : : +- ReusedSubquery - : : : : : +- BroadcastExchange - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- Filter - : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- ReusedSubquery - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.item - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- CometNativeColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.date_dim - :- HashAggregate - : +- CometNativeColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- HashAggregate - : +- CometNativeColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Union - : :- Filter - : : : +- Subquery - : : : +- HashAggregate - : : : +- CometNativeColumnarToRow - : : : +- CometColumnarExchange - : : : +- HashAggregate - : : : +- Union - : : : :- Project - : : : : +- BroadcastHashJoin - : : : : :- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : +- ReusedSubquery - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : :- Project - : : : : +- BroadcastHashJoin - : : : : :- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- Project - : : : +- BroadcastHashJoin - : : : :- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- ReusedSubquery - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- HashAggregate - : : +- CometNativeColumnarToRow - : : +- CometColumnarExchange - : : +- HashAggregate - : : +- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- BroadcastHashJoin - : : : : :- Filter - : : : : : +- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : +- BroadcastExchange - : : : : +- Project - : : : : +- BroadcastHashJoin - : : : : :- CometNativeColumnarToRow - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : +- BroadcastExchange - : : : : +- BroadcastHashJoin - : : : : :- CometNativeColumnarToRow - : : : : : +- CometHashAggregate - : : : : : +- CometColumnarExchange - : : : : : +- HashAggregate - : : : : : +- Project - : : : : : +- BroadcastHashJoin - : : : : : :- Project - : : : : : : +- BroadcastHashJoin - : : : : : : :- Filter - : : : : : : : +- ColumnarToRow - : : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : : : +- SubqueryBroadcast - : : : : : : : +- BroadcastExchange - : : : : : : : +- CometNativeColumnarToRow - : : : : : : : +- CometProject - : : : : : : : +- CometFilter - : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : : : +- BroadcastExchange - : : : : : : +- BroadcastHashJoin - : : : : : : :- CometNativeColumnarToRow - : : : : : : : +- CometFilter - : : : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : : : +- BroadcastExchange - : : : : : : +- Project - : : : : : : +- BroadcastHashJoin - : : : : : : :- Project - : : : : : : : +- BroadcastHashJoin - : : : : : : : :- Filter - : : : : : : : : +- ColumnarToRow - : : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : : : : +- ReusedSubquery - : : : : : : : +- BroadcastExchange - : : : : : : : +- CometNativeColumnarToRow - : : : : : : : +- CometFilter - : : : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : : : +- BroadcastExchange - : : : : : : +- CometNativeColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : : +- BroadcastExchange - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : +- BroadcastExchange - : : : : +- Project - : : : : +- BroadcastHashJoin - : : : : :- Project - : : : : : +- BroadcastHashJoin - : : : : : :- Filter - : : : : : : +- ColumnarToRow - : : : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : : +- ReusedSubquery - : : : : : +- BroadcastExchange - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- BroadcastHashJoin - : : : :- CometNativeColumnarToRow - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : +- BroadcastExchange - : : : +- Project - : : : +- BroadcastHashJoin - : : : :- CometNativeColumnarToRow - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : +- BroadcastExchange - : : : +- BroadcastHashJoin - : : : :- CometNativeColumnarToRow - : : : : +- CometHashAggregate - : : : : +- CometColumnarExchange - : : : : +- HashAggregate - : : : : +- Project - : : : : +- BroadcastHashJoin - : : : : :- Project - : : : : : +- BroadcastHashJoin - : : : : : :- Filter - : : : : : : +- ColumnarToRow - : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometNativeColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : : +- BroadcastExchange - : : : : : +- BroadcastHashJoin - : : : : : :- CometNativeColumnarToRow - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : : +- BroadcastExchange - : : : : : +- Project - : : : : : +- BroadcastHashJoin - : : : : : :- Project - : : : : : : +- BroadcastHashJoin - : : : : : : :- Filter - : : : : : : : +- ColumnarToRow - : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : : : +- ReusedSubquery - : : : : : : +- BroadcastExchange - : : : : : : +- CometNativeColumnarToRow - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : : +- BroadcastExchange - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- Project - : : : +- BroadcastHashJoin - : : : :- Project - : : : : +- BroadcastHashJoin - : : : : :- Filter - : : : : : +- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : +- ReusedSubquery - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : :- Filter - : : : +- ReusedSubquery - : : +- HashAggregate - : : +- CometNativeColumnarToRow - : : +- CometColumnarExchange - : : +- HashAggregate - : : +- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- BroadcastHashJoin - : : : : :- Filter - : : : : : +- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : +- ReusedSubquery - : : : : +- BroadcastExchange - : : : : +- Project - : : : : +- BroadcastHashJoin - : : : : :- CometNativeColumnarToRow - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : +- BroadcastExchange - : : : : +- BroadcastHashJoin - : : : : :- CometNativeColumnarToRow - : : : : : +- CometHashAggregate - : : : : : +- CometColumnarExchange - : : : : : +- HashAggregate - : : : : : +- Project - : : : : : +- BroadcastHashJoin - : : : : : :- Project - : : : : : : +- BroadcastHashJoin - : : : : : : :- Filter - : : : : : : : +- ColumnarToRow - : : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : : : +- SubqueryBroadcast - : : : : : : : +- BroadcastExchange - : : : : : : : +- CometNativeColumnarToRow - : : : : : : : +- CometProject - : : : : : : : +- CometFilter - : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : : : +- BroadcastExchange - : : : : : : +- BroadcastHashJoin - : : : : : : :- CometNativeColumnarToRow - : : : : : : : +- CometFilter - : : : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : : : +- BroadcastExchange - : : : : : : +- Project - : : : : : : +- BroadcastHashJoin - : : : : : : :- Project - : : : : : : : +- BroadcastHashJoin - : : : : : : : :- Filter - : : : : : : : : +- ColumnarToRow - : : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : : : : +- ReusedSubquery - : : : : : : : +- BroadcastExchange - : : : : : : : +- CometNativeColumnarToRow - : : : : : : : +- CometFilter - : : : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : : : +- BroadcastExchange - : : : : : : +- CometNativeColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : : +- BroadcastExchange - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : +- BroadcastExchange - : : : : +- Project - : : : : +- BroadcastHashJoin - : : : : :- Project - : : : : : +- BroadcastHashJoin - : : : : : :- Filter - : : : : : : +- ColumnarToRow - : : : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : : +- ReusedSubquery - : : : : : +- BroadcastExchange - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- BroadcastHashJoin - : : : :- CometNativeColumnarToRow - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : +- BroadcastExchange - : : : +- Project - : : : +- BroadcastHashJoin - : : : :- CometNativeColumnarToRow - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : +- BroadcastExchange - : : : +- BroadcastHashJoin - : : : :- CometNativeColumnarToRow - : : : : +- CometHashAggregate - : : : : +- CometColumnarExchange - : : : : +- HashAggregate - : : : : +- Project - : : : : +- BroadcastHashJoin - : : : : :- Project - : : : : : +- BroadcastHashJoin - : : : : : :- Filter - : : : : : : +- ColumnarToRow - : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometNativeColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : : +- BroadcastExchange - : : : : : +- BroadcastHashJoin - : : : : : :- CometNativeColumnarToRow - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : : +- BroadcastExchange - : : : : : +- Project - : : : : : +- BroadcastHashJoin - : : : : : :- Project - : : : : : : +- BroadcastHashJoin - : : : : : : :- Filter - : : : : : : : +- ColumnarToRow - : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : : : +- ReusedSubquery - : : : : : : +- BroadcastExchange - : : : : : : +- CometNativeColumnarToRow - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : : +- BroadcastExchange - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- Project - : : : +- BroadcastHashJoin - : : : :- Project - : : : : +- BroadcastHashJoin - : : : : :- Filter - : : : : : +- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : +- ReusedSubquery - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- Filter - : : +- ReusedSubquery - : +- HashAggregate - : +- CometNativeColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- BroadcastHashJoin - : : : :- Filter - : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- ReusedSubquery - : : : +- BroadcastExchange - : : : +- Project - : : : +- BroadcastHashJoin - : : : :- CometNativeColumnarToRow - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : +- BroadcastExchange - : : : +- BroadcastHashJoin - : : : :- CometNativeColumnarToRow - : : : : +- CometHashAggregate - : : : : +- CometColumnarExchange - : : : : +- HashAggregate - : : : : +- Project - : : : : +- BroadcastHashJoin - : : : : :- Project - : : : : : +- BroadcastHashJoin - : : : : : :- Filter - : : : : : : +- ColumnarToRow - : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometNativeColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : : +- BroadcastExchange - : : : : : +- BroadcastHashJoin - : : : : : :- CometNativeColumnarToRow - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : : +- BroadcastExchange - : : : : : +- Project - : : : : : +- BroadcastHashJoin - : : : : : :- Project - : : : : : : +- BroadcastHashJoin - : : : : : : :- Filter - : : : : : : : +- ColumnarToRow - : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : : : +- ReusedSubquery - : : : : : : +- BroadcastExchange - : : : : : : +- CometNativeColumnarToRow - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : : +- BroadcastExchange - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- Project - : : : +- BroadcastHashJoin - : : : :- Project - : : : : +- BroadcastHashJoin - : : : : :- Filter - : : : : : +- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : +- ReusedSubquery - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- BroadcastHashJoin - : : :- CometNativeColumnarToRow - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.item - : : +- BroadcastExchange - : : +- Project - : : +- BroadcastHashJoin - : : :- CometNativeColumnarToRow - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.item - : : +- BroadcastExchange - : : +- BroadcastHashJoin - : : :- CometNativeColumnarToRow - : : : +- CometHashAggregate - : : : +- CometColumnarExchange - : : : +- HashAggregate - : : : +- Project - : : : +- BroadcastHashJoin - : : : :- Project - : : : : +- BroadcastHashJoin - : : : : :- Filter - : : : : : +- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : +- BroadcastExchange - : : : : +- BroadcastHashJoin - : : : : :- CometNativeColumnarToRow - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : +- BroadcastExchange - : : : : +- Project - : : : : +- BroadcastHashJoin - : : : : :- Project - : : : : : +- BroadcastHashJoin - : : : : : :- Filter - : : : : : : +- ColumnarToRow - : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : : +- ReusedSubquery - : : : : : +- BroadcastExchange - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- Filter - : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- ReusedSubquery - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.item - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- CometNativeColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.date_dim - +- HashAggregate - +- CometNativeColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- HashAggregate - +- CometNativeColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Union - :- Filter - : : +- Subquery - : : +- HashAggregate - : : +- CometNativeColumnarToRow - : : +- CometColumnarExchange - : : +- HashAggregate - : : +- Union - : : :- Project - : : : +- BroadcastHashJoin - : : : :- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- ReusedSubquery - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : :- Project - : : : +- BroadcastHashJoin - : : : :- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- Project - : : +- BroadcastHashJoin - : : :- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : +- ReusedSubquery - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- HashAggregate - : +- CometNativeColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- BroadcastHashJoin - : : : :- Filter - : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- Project - : : : +- BroadcastHashJoin - : : : :- CometNativeColumnarToRow - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : +- BroadcastExchange - : : : +- BroadcastHashJoin - : : : :- CometNativeColumnarToRow - : : : : +- CometHashAggregate - : : : : +- CometColumnarExchange - : : : : +- HashAggregate - : : : : +- Project - : : : : +- BroadcastHashJoin - : : : : :- Project - : : : : : +- BroadcastHashJoin - : : : : : :- Filter - : : : : : : +- ColumnarToRow - : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometNativeColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : : +- BroadcastExchange - : : : : : +- BroadcastHashJoin - : : : : : :- CometNativeColumnarToRow - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : : +- BroadcastExchange - : : : : : +- Project - : : : : : +- BroadcastHashJoin - : : : : : :- Project - : : : : : : +- BroadcastHashJoin - : : : : : : :- Filter - : : : : : : : +- ColumnarToRow - : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : : : +- ReusedSubquery - : : : : : : +- BroadcastExchange - : : : : : : +- CometNativeColumnarToRow - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : : +- BroadcastExchange - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- Project - : : : +- BroadcastHashJoin - : : : :- Project - : : : : +- BroadcastHashJoin - : : : : :- Filter - : : : : : +- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : +- ReusedSubquery - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- BroadcastHashJoin - : : :- CometNativeColumnarToRow - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.item - : : +- BroadcastExchange - : : +- Project - : : +- BroadcastHashJoin - : : :- CometNativeColumnarToRow - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.item - : : +- BroadcastExchange - : : +- BroadcastHashJoin - : : :- CometNativeColumnarToRow - : : : +- CometHashAggregate - : : : +- CometColumnarExchange - : : : +- HashAggregate - : : : +- Project - : : : +- BroadcastHashJoin - : : : :- Project - : : : : +- BroadcastHashJoin - : : : : :- Filter - : : : : : +- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : +- BroadcastExchange - : : : : +- BroadcastHashJoin - : : : : :- CometNativeColumnarToRow - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : +- BroadcastExchange - : : : : +- Project - : : : : +- BroadcastHashJoin - : : : : :- Project - : : : : : +- BroadcastHashJoin - : : : : : :- Filter - : : : : : : +- ColumnarToRow - : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : : +- ReusedSubquery - : : : : : +- BroadcastExchange - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- Filter - : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- ReusedSubquery - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.item - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- CometNativeColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.date_dim - :- Filter - : : +- ReusedSubquery - : +- HashAggregate - : +- CometNativeColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- BroadcastHashJoin - : : : :- Filter - : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- ReusedSubquery - : : : +- BroadcastExchange - : : : +- Project - : : : +- BroadcastHashJoin - : : : :- CometNativeColumnarToRow - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : +- BroadcastExchange - : : : +- BroadcastHashJoin - : : : :- CometNativeColumnarToRow - : : : : +- CometHashAggregate - : : : : +- CometColumnarExchange - : : : : +- HashAggregate - : : : : +- Project - : : : : +- BroadcastHashJoin - : : : : :- Project - : : : : : +- BroadcastHashJoin - : : : : : :- Filter - : : : : : : +- ColumnarToRow - : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometNativeColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : : +- BroadcastExchange - : : : : : +- BroadcastHashJoin - : : : : : :- CometNativeColumnarToRow - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : : +- BroadcastExchange - : : : : : +- Project - : : : : : +- BroadcastHashJoin - : : : : : :- Project - : : : : : : +- BroadcastHashJoin - : : : : : : :- Filter - : : : : : : : +- ColumnarToRow - : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : : : +- ReusedSubquery - : : : : : : +- BroadcastExchange - : : : : : : +- CometNativeColumnarToRow - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : : +- BroadcastExchange - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- Project - : : : +- BroadcastHashJoin - : : : :- Project - : : : : +- BroadcastHashJoin - : : : : :- Filter - : : : : : +- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : +- ReusedSubquery - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- BroadcastHashJoin - : : :- CometNativeColumnarToRow - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.item - : : +- BroadcastExchange - : : +- Project - : : +- BroadcastHashJoin - : : :- CometNativeColumnarToRow - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.item - : : +- BroadcastExchange - : : +- BroadcastHashJoin - : : :- CometNativeColumnarToRow - : : : +- CometHashAggregate - : : : +- CometColumnarExchange - : : : +- HashAggregate - : : : +- Project - : : : +- BroadcastHashJoin - : : : :- Project - : : : : +- BroadcastHashJoin - : : : : :- Filter - : : : : : +- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : +- BroadcastExchange - : : : : +- BroadcastHashJoin - : : : : :- CometNativeColumnarToRow - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : +- BroadcastExchange - : : : : +- Project - : : : : +- BroadcastHashJoin - : : : : :- Project - : : : : : +- BroadcastHashJoin - : : : : : :- Filter - : : : : : : +- ColumnarToRow - : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : : +- ReusedSubquery - : : : : : +- BroadcastExchange - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- Filter - : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- ReusedSubquery - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.item - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- CometNativeColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.date_dim - +- Filter - : +- ReusedSubquery - +- HashAggregate - +- CometNativeColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- BroadcastHashJoin - : : :- Filter - : : : +- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : +- ReusedSubquery - : : +- BroadcastExchange - : : +- Project - : : +- BroadcastHashJoin - : : :- CometNativeColumnarToRow - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.item - : : +- BroadcastExchange - : : +- BroadcastHashJoin - : : :- CometNativeColumnarToRow - : : : +- CometHashAggregate - : : : +- CometColumnarExchange - : : : +- HashAggregate - : : : +- Project - : : : +- BroadcastHashJoin - : : : :- Project - : : : : +- BroadcastHashJoin - : : : : :- Filter - : : : : : +- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : +- BroadcastExchange - : : : : +- BroadcastHashJoin - : : : : :- CometNativeColumnarToRow - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : +- BroadcastExchange - : : : : +- Project - : : : : +- BroadcastHashJoin - : : : : :- Project - : : : : : +- BroadcastHashJoin - : : : : : :- Filter - : : : : : : +- ColumnarToRow - : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : : +- ReusedSubquery - : : : : : +- BroadcastExchange - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- Filter - : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- ReusedSubquery - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.item - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- BroadcastHashJoin - : :- CometNativeColumnarToRow - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.item - : +- BroadcastExchange - : +- Project - : +- BroadcastHashJoin - : :- CometNativeColumnarToRow - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.item - : +- BroadcastExchange - : +- BroadcastHashJoin - : :- CometNativeColumnarToRow - : : +- CometHashAggregate - : : +- CometColumnarExchange - : : +- HashAggregate - : : +- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- Filter - : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- BroadcastHashJoin - : : : :- CometNativeColumnarToRow - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : +- BroadcastExchange - : : : +- Project - : : : +- BroadcastHashJoin - : : : :- Project - : : : : +- BroadcastHashJoin - : : : : :- Filter - : : : : : +- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : +- ReusedSubquery - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Filter - : : : +- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : +- ReusedSubquery - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.item - : +- BroadcastExchange - : +- CometNativeColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.date_dim - +- BroadcastExchange - +- CometNativeColumnarToRow - +- CometProject - +- CometFilter - +- CometNativeScan parquet spark_catalog.default.date_dim + :- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometUnion + : :- CometFilter + : : : +- Subquery + : : : +- CometNativeColumnarToRow + : : : +- CometHashAggregate + : : : +- CometExchange + : : : +- CometHashAggregate + : : : +- CometUnion + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometNativeScan parquet spark_catalog.default.store_sales + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometNativeColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometNativeColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometNativeScan parquet spark_catalog.default.web_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometNativeColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometHashAggregate + : : +- CometExchange + : : +- CometHashAggregate + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometBroadcastHashJoin + : : : : :- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometNativeColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : : +- CometBroadcastExchange + : : : : +- CometBroadcastHashJoin + : : : : :- CometHashAggregate + : : : : : +- CometExchange + : : : : : +- CometHashAggregate + : : : : : +- CometProject + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometProject + : : : : : : +- CometBroadcastHashJoin + : : : : : : :- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : : : : +- SubqueryBroadcast + : : : : : : : +- BroadcastExchange + : : : : : : : +- CometNativeColumnarToRow + : : : : : : : +- CometProject + : : : : : : : +- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : +- CometBroadcastExchange + : : : : : : +- CometBroadcastHashJoin + : : : : : : :- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : : : : +- CometBroadcastExchange + : : : : : : +- CometProject + : : : : : : +- CometBroadcastHashJoin + : : : : : : :- CometProject + : : : : : : : +- CometBroadcastHashJoin + : : : : : : : :- CometFilter + : : : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : : : : : +- SubqueryBroadcast + : : : : : : : : +- BroadcastExchange + : : : : : : : : +- CometNativeColumnarToRow + : : : : : : : : +- CometProject + : : : : : : : : +- CometFilter + : : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : : +- CometBroadcastExchange + : : : : : : : +- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : : : : +- CometBroadcastExchange + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- CometBroadcastExchange + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometProject + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.web_sales + : : : : : : +- SubqueryBroadcast + : : : : : : +- BroadcastExchange + : : : : : : +- CometNativeColumnarToRow + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- CometBroadcastExchange + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : +- CometBroadcastExchange + : : : +- CometBroadcastHashJoin + : : : :- CometHashAggregate + : : : : +- CometExchange + : : : : +- CometHashAggregate + : : : : +- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometProject + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : : : +- SubqueryBroadcast + : : : : : : +- BroadcastExchange + : : : : : : +- CometNativeColumnarToRow + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- CometBroadcastExchange + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : : : +- CometBroadcastExchange + : : : : : +- CometProject + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometProject + : : : : : : +- CometBroadcastHashJoin + : : : : : : :- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : : : : +- SubqueryBroadcast + : : : : : : : +- BroadcastExchange + : : : : : : : +- CometNativeColumnarToRow + : : : : : : : +- CometProject + : : : : : : : +- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : +- CometBroadcastExchange + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : : : +- CometBroadcastExchange + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.web_sales + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometNativeColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : :- CometFilter + : : : +- ReusedSubquery + : : +- CometHashAggregate + : : +- CometExchange + : : +- CometHashAggregate + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometBroadcastHashJoin + : : : : :- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometNativeColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : : +- CometBroadcastExchange + : : : : +- CometBroadcastHashJoin + : : : : :- CometHashAggregate + : : : : : +- CometExchange + : : : : : +- CometHashAggregate + : : : : : +- CometProject + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometProject + : : : : : : +- CometBroadcastHashJoin + : : : : : : :- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : : : : +- SubqueryBroadcast + : : : : : : : +- BroadcastExchange + : : : : : : : +- CometNativeColumnarToRow + : : : : : : : +- CometProject + : : : : : : : +- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : +- CometBroadcastExchange + : : : : : : +- CometBroadcastHashJoin + : : : : : : :- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : : : : +- CometBroadcastExchange + : : : : : : +- CometProject + : : : : : : +- CometBroadcastHashJoin + : : : : : : :- CometProject + : : : : : : : +- CometBroadcastHashJoin + : : : : : : : :- CometFilter + : : : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : : : : : +- SubqueryBroadcast + : : : : : : : : +- BroadcastExchange + : : : : : : : : +- CometNativeColumnarToRow + : : : : : : : : +- CometProject + : : : : : : : : +- CometFilter + : : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : : +- CometBroadcastExchange + : : : : : : : +- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : : : : +- CometBroadcastExchange + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- CometBroadcastExchange + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometProject + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.web_sales + : : : : : : +- SubqueryBroadcast + : : : : : : +- BroadcastExchange + : : : : : : +- CometNativeColumnarToRow + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- CometBroadcastExchange + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : +- CometBroadcastExchange + : : : +- CometBroadcastHashJoin + : : : :- CometHashAggregate + : : : : +- CometExchange + : : : : +- CometHashAggregate + : : : : +- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometProject + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : : : +- SubqueryBroadcast + : : : : : : +- BroadcastExchange + : : : : : : +- CometNativeColumnarToRow + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- CometBroadcastExchange + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : : : +- CometBroadcastExchange + : : : : : +- CometProject + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometProject + : : : : : : +- CometBroadcastHashJoin + : : : : : : :- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : : : : +- SubqueryBroadcast + : : : : : : : +- BroadcastExchange + : : : : : : : +- CometNativeColumnarToRow + : : : : : : : +- CometProject + : : : : : : : +- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : +- CometBroadcastExchange + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : : : +- CometBroadcastExchange + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.web_sales + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometNativeColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometFilter + : : +- ReusedSubquery + : +- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.web_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometNativeColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : +- CometBroadcastExchange + : : : +- CometBroadcastHashJoin + : : : :- CometHashAggregate + : : : : +- CometExchange + : : : : +- CometHashAggregate + : : : : +- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometProject + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : : : +- SubqueryBroadcast + : : : : : : +- BroadcastExchange + : : : : : : +- CometNativeColumnarToRow + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- CometBroadcastExchange + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : : : +- CometBroadcastExchange + : : : : : +- CometProject + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometProject + : : : : : : +- CometBroadcastHashJoin + : : : : : : :- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : : : : +- SubqueryBroadcast + : : : : : : : +- BroadcastExchange + : : : : : : : +- CometNativeColumnarToRow + : : : : : : : +- CometProject + : : : : : : : +- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : +- CometBroadcastExchange + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : : : +- CometBroadcastExchange + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.web_sales + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometNativeColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.item + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.item + : : +- CometBroadcastExchange + : : +- CometBroadcastHashJoin + : : :- CometHashAggregate + : : : +- CometExchange + : : : +- CometHashAggregate + : : : +- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometNativeColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometBroadcastHashJoin + : : : : :- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometProject + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : : : +- SubqueryBroadcast + : : : : : : +- BroadcastExchange + : : : : : : +- CometNativeColumnarToRow + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- CometBroadcastExchange + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.web_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometNativeColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.item + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.date_dim + :- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometUnion + : :- CometFilter + : : : +- Subquery + : : : +- CometNativeColumnarToRow + : : : +- CometHashAggregate + : : : +- CometExchange + : : : +- CometHashAggregate + : : : +- CometUnion + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometNativeScan parquet spark_catalog.default.store_sales + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometNativeColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometNativeColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometNativeScan parquet spark_catalog.default.web_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometNativeColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometHashAggregate + : : +- CometExchange + : : +- CometHashAggregate + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometBroadcastHashJoin + : : : : :- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometNativeColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : : +- CometBroadcastExchange + : : : : +- CometBroadcastHashJoin + : : : : :- CometHashAggregate + : : : : : +- CometExchange + : : : : : +- CometHashAggregate + : : : : : +- CometProject + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometProject + : : : : : : +- CometBroadcastHashJoin + : : : : : : :- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : : : : +- SubqueryBroadcast + : : : : : : : +- BroadcastExchange + : : : : : : : +- CometNativeColumnarToRow + : : : : : : : +- CometProject + : : : : : : : +- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : +- CometBroadcastExchange + : : : : : : +- CometBroadcastHashJoin + : : : : : : :- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : : : : +- CometBroadcastExchange + : : : : : : +- CometProject + : : : : : : +- CometBroadcastHashJoin + : : : : : : :- CometProject + : : : : : : : +- CometBroadcastHashJoin + : : : : : : : :- CometFilter + : : : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : : : : : +- SubqueryBroadcast + : : : : : : : : +- BroadcastExchange + : : : : : : : : +- CometNativeColumnarToRow + : : : : : : : : +- CometProject + : : : : : : : : +- CometFilter + : : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : : +- CometBroadcastExchange + : : : : : : : +- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : : : : +- CometBroadcastExchange + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- CometBroadcastExchange + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometProject + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.web_sales + : : : : : : +- SubqueryBroadcast + : : : : : : +- BroadcastExchange + : : : : : : +- CometNativeColumnarToRow + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- CometBroadcastExchange + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : +- CometBroadcastExchange + : : : +- CometBroadcastHashJoin + : : : :- CometHashAggregate + : : : : +- CometExchange + : : : : +- CometHashAggregate + : : : : +- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometProject + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : : : +- SubqueryBroadcast + : : : : : : +- BroadcastExchange + : : : : : : +- CometNativeColumnarToRow + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- CometBroadcastExchange + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : : : +- CometBroadcastExchange + : : : : : +- CometProject + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometProject + : : : : : : +- CometBroadcastHashJoin + : : : : : : :- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : : : : +- SubqueryBroadcast + : : : : : : : +- BroadcastExchange + : : : : : : : +- CometNativeColumnarToRow + : : : : : : : +- CometProject + : : : : : : : +- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : +- CometBroadcastExchange + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : : : +- CometBroadcastExchange + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.web_sales + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometNativeColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : :- CometFilter + : : : +- ReusedSubquery + : : +- CometHashAggregate + : : +- CometExchange + : : +- CometHashAggregate + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometBroadcastHashJoin + : : : : :- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometNativeColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : : +- CometBroadcastExchange + : : : : +- CometBroadcastHashJoin + : : : : :- CometHashAggregate + : : : : : +- CometExchange + : : : : : +- CometHashAggregate + : : : : : +- CometProject + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometProject + : : : : : : +- CometBroadcastHashJoin + : : : : : : :- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : : : : +- SubqueryBroadcast + : : : : : : : +- BroadcastExchange + : : : : : : : +- CometNativeColumnarToRow + : : : : : : : +- CometProject + : : : : : : : +- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : +- CometBroadcastExchange + : : : : : : +- CometBroadcastHashJoin + : : : : : : :- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : : : : +- CometBroadcastExchange + : : : : : : +- CometProject + : : : : : : +- CometBroadcastHashJoin + : : : : : : :- CometProject + : : : : : : : +- CometBroadcastHashJoin + : : : : : : : :- CometFilter + : : : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : : : : : +- SubqueryBroadcast + : : : : : : : : +- BroadcastExchange + : : : : : : : : +- CometNativeColumnarToRow + : : : : : : : : +- CometProject + : : : : : : : : +- CometFilter + : : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : : +- CometBroadcastExchange + : : : : : : : +- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : : : : +- CometBroadcastExchange + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- CometBroadcastExchange + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometProject + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.web_sales + : : : : : : +- SubqueryBroadcast + : : : : : : +- BroadcastExchange + : : : : : : +- CometNativeColumnarToRow + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- CometBroadcastExchange + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : +- CometBroadcastExchange + : : : +- CometBroadcastHashJoin + : : : :- CometHashAggregate + : : : : +- CometExchange + : : : : +- CometHashAggregate + : : : : +- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometProject + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : : : +- SubqueryBroadcast + : : : : : : +- BroadcastExchange + : : : : : : +- CometNativeColumnarToRow + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- CometBroadcastExchange + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : : : +- CometBroadcastExchange + : : : : : +- CometProject + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometProject + : : : : : : +- CometBroadcastHashJoin + : : : : : : :- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : : : : +- SubqueryBroadcast + : : : : : : : +- BroadcastExchange + : : : : : : : +- CometNativeColumnarToRow + : : : : : : : +- CometProject + : : : : : : : +- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : +- CometBroadcastExchange + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : : : +- CometBroadcastExchange + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.web_sales + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometNativeColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometFilter + : : +- ReusedSubquery + : +- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.web_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometNativeColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : +- CometBroadcastExchange + : : : +- CometBroadcastHashJoin + : : : :- CometHashAggregate + : : : : +- CometExchange + : : : : +- CometHashAggregate + : : : : +- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometProject + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : : : +- SubqueryBroadcast + : : : : : : +- BroadcastExchange + : : : : : : +- CometNativeColumnarToRow + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- CometBroadcastExchange + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : : : +- CometBroadcastExchange + : : : : : +- CometProject + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometProject + : : : : : : +- CometBroadcastHashJoin + : : : : : : :- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : : : : +- SubqueryBroadcast + : : : : : : : +- BroadcastExchange + : : : : : : : +- CometNativeColumnarToRow + : : : : : : : +- CometProject + : : : : : : : +- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : +- CometBroadcastExchange + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : : : +- CometBroadcastExchange + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.web_sales + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometNativeColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.item + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.item + : : +- CometBroadcastExchange + : : +- CometBroadcastHashJoin + : : :- CometHashAggregate + : : : +- CometExchange + : : : +- CometHashAggregate + : : : +- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometNativeColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometBroadcastHashJoin + : : : : :- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometProject + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : : : +- SubqueryBroadcast + : : : : : : +- BroadcastExchange + : : : : : : +- CometNativeColumnarToRow + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- CometBroadcastExchange + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.web_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometNativeColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.item + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.date_dim + +- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometUnion + :- CometFilter + : : +- Subquery + : : +- CometNativeColumnarToRow + : : +- CometHashAggregate + : : +- CometExchange + : : +- CometHashAggregate + : : +- CometUnion + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometNativeScan parquet spark_catalog.default.store_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometNativeColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometNativeColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometNativeScan parquet spark_catalog.default.web_sales + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometNativeColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometNativeColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : +- CometBroadcastExchange + : : : +- CometBroadcastHashJoin + : : : :- CometHashAggregate + : : : : +- CometExchange + : : : : +- CometHashAggregate + : : : : +- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometProject + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : : : +- SubqueryBroadcast + : : : : : : +- BroadcastExchange + : : : : : : +- CometNativeColumnarToRow + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- CometBroadcastExchange + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : : : +- CometBroadcastExchange + : : : : : +- CometProject + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometProject + : : : : : : +- CometBroadcastHashJoin + : : : : : : :- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : : : : +- SubqueryBroadcast + : : : : : : : +- BroadcastExchange + : : : : : : : +- CometNativeColumnarToRow + : : : : : : : +- CometProject + : : : : : : : +- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : +- CometBroadcastExchange + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : : : +- CometBroadcastExchange + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.web_sales + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometNativeColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.item + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.item + : : +- CometBroadcastExchange + : : +- CometBroadcastHashJoin + : : :- CometHashAggregate + : : : +- CometExchange + : : : +- CometHashAggregate + : : : +- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometNativeColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometBroadcastHashJoin + : : : : :- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometProject + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : : : +- SubqueryBroadcast + : : : : : : +- BroadcastExchange + : : : : : : +- CometNativeColumnarToRow + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- CometBroadcastExchange + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.web_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometNativeColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.item + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.date_dim + :- CometFilter + : : +- ReusedSubquery + : +- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometNativeColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : +- CometBroadcastExchange + : : : +- CometBroadcastHashJoin + : : : :- CometHashAggregate + : : : : +- CometExchange + : : : : +- CometHashAggregate + : : : : +- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometProject + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : : : +- SubqueryBroadcast + : : : : : : +- BroadcastExchange + : : : : : : +- CometNativeColumnarToRow + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- CometBroadcastExchange + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : : : +- CometBroadcastExchange + : : : : : +- CometProject + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometProject + : : : : : : +- CometBroadcastHashJoin + : : : : : : :- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : : : : +- SubqueryBroadcast + : : : : : : : +- BroadcastExchange + : : : : : : : +- CometNativeColumnarToRow + : : : : : : : +- CometProject + : : : : : : : +- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : +- CometBroadcastExchange + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : : : +- CometBroadcastExchange + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.web_sales + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometNativeColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.item + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.item + : : +- CometBroadcastExchange + : : +- CometBroadcastHashJoin + : : :- CometHashAggregate + : : : +- CometExchange + : : : +- CometHashAggregate + : : : +- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometNativeColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometBroadcastHashJoin + : : : : :- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometProject + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : : : +- SubqueryBroadcast + : : : : : : +- BroadcastExchange + : : : : : : +- CometNativeColumnarToRow + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- CometBroadcastExchange + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.web_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometNativeColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.item + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.date_dim + +- CometFilter + : +- ReusedSubquery + +- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.web_sales + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometNativeColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.item + : : +- CometBroadcastExchange + : : +- CometBroadcastHashJoin + : : :- CometHashAggregate + : : : +- CometExchange + : : : +- CometHashAggregate + : : : +- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometNativeColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometBroadcastHashJoin + : : : : :- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometProject + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : : : +- SubqueryBroadcast + : : : : : : +- BroadcastExchange + : : : : : : +- CometNativeColumnarToRow + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- CometBroadcastExchange + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.web_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometNativeColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.item + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometBroadcastHashJoin + : :- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.item + : +- CometBroadcastExchange + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.item + : +- CometBroadcastExchange + : +- CometBroadcastHashJoin + : :- CometHashAggregate + : : +- CometExchange + : : +- CometHashAggregate + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometNativeColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometNativeColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.web_sales + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometNativeColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.item + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.date_dim + +- CometBroadcastExchange + +- CometProject + +- CometFilter + +- CometNativeScan parquet spark_catalog.default.date_dim -Comet accelerated 842 out of 2302 eligible operators (36%). Final plan contains 475 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 2367 out of 2622 eligible operators (90%). Final plan contains 126 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q18a.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q18a.native_datafusion/extended.txt index 8ad4e890ec..82bacea2fb 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q18a.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q18a.native_datafusion/extended.txt @@ -1,253 +1,234 @@ -TakeOrderedAndProject -+- Union - :- HashAggregate - : +- CometNativeColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- Project - : : : : +- BroadcastHashJoin - : : : : :- Project - : : : : : +- BroadcastHashJoin - : : : : : :- Project - : : : : : : +- BroadcastHashJoin - : : : : : : :- Filter - : : : : : : : +- ColumnarToRow - : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : : : +- SubqueryBroadcast - : : : : : : : +- BroadcastExchange - : : : : : : : +- CometNativeColumnarToRow - : : : : : : : +- CometProject - : : : : : : : +- CometFilter - : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : : : +- BroadcastExchange - : : : : : : +- CometNativeColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.customer_demographics - : : : : : +- BroadcastExchange - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.customer - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.customer_demographics - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.customer_address - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- CometNativeColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.item - :- HashAggregate - : +- CometNativeColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- Project - : : : : +- BroadcastHashJoin - : : : : :- Project - : : : : : +- BroadcastHashJoin - : : : : : :- Project - : : : : : : +- BroadcastHashJoin - : : : : : : :- Filter - : : : : : : : +- ColumnarToRow - : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : : : +- ReusedSubquery - : : : : : : +- BroadcastExchange - : : : : : : +- CometNativeColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.customer_demographics - : : : : : +- BroadcastExchange - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.customer - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.customer_demographics - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.customer_address - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- CometNativeColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.item - :- HashAggregate - : +- CometNativeColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- Project - : : : : +- BroadcastHashJoin - : : : : :- Project - : : : : : +- BroadcastHashJoin - : : : : : :- Project - : : : : : : +- BroadcastHashJoin - : : : : : : :- Filter - : : : : : : : +- ColumnarToRow - : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : : : +- ReusedSubquery - : : : : : : +- BroadcastExchange - : : : : : : +- CometNativeColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.customer_demographics - : : : : : +- BroadcastExchange - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.customer - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.customer_demographics - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.customer_address - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- CometNativeColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.item - :- HashAggregate - : +- CometNativeColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- Project - : : : : +- BroadcastHashJoin - : : : : :- Project - : : : : : +- BroadcastHashJoin - : : : : : :- Project - : : : : : : +- BroadcastHashJoin - : : : : : : :- Filter - : : : : : : : +- ColumnarToRow - : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : : : +- ReusedSubquery - : : : : : : +- BroadcastExchange - : : : : : : +- CometNativeColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.customer_demographics - : : : : : +- BroadcastExchange - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.customer - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.customer_demographics - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.customer_address - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- CometNativeColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.item - +- HashAggregate - +- CometNativeColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- Project - : : : : +- BroadcastHashJoin - : : : : :- Project - : : : : : +- BroadcastHashJoin - : : : : : :- Filter - : : : : : : +- ColumnarToRow - : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : : +- ReusedSubquery - : : : : : +- BroadcastExchange - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.customer_demographics - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.customer - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.customer_demographics - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.customer_address - : +- BroadcastExchange - : +- CometNativeColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.date_dim - +- BroadcastExchange - +- CometNativeColumnarToRow - +- CometFilter - +- CometNativeScan parquet spark_catalog.default.item +CometNativeColumnarToRow ++- CometTakeOrderedAndProject + +- CometUnion + :- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometProject + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometProject + : : : : : : +- CometBroadcastHashJoin + : : : : : : :- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : : : : +- SubqueryBroadcast + : : : : : : : +- BroadcastExchange + : : : : : : : +- CometNativeColumnarToRow + : : : : : : : +- CometProject + : : : : : : : +- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : +- CometBroadcastExchange + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.customer_demographics + : : : : : +- CometBroadcastExchange + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.customer + : : : : +- CometBroadcastExchange + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.customer_demographics + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.customer_address + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.item + :- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometProject + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometProject + : : : : : : +- CometBroadcastHashJoin + : : : : : : :- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : : : : +- SubqueryBroadcast + : : : : : : : +- BroadcastExchange + : : : : : : : +- CometNativeColumnarToRow + : : : : : : : +- CometProject + : : : : : : : +- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : +- CometBroadcastExchange + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.customer_demographics + : : : : : +- CometBroadcastExchange + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.customer + : : : : +- CometBroadcastExchange + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.customer_demographics + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.customer_address + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.item + :- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometProject + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometProject + : : : : : : +- CometBroadcastHashJoin + : : : : : : :- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : : : : +- SubqueryBroadcast + : : : : : : : +- BroadcastExchange + : : : : : : : +- CometNativeColumnarToRow + : : : : : : : +- CometProject + : : : : : : : +- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : +- CometBroadcastExchange + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.customer_demographics + : : : : : +- CometBroadcastExchange + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.customer + : : : : +- CometBroadcastExchange + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.customer_demographics + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.customer_address + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.item + :- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometProject + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometProject + : : : : : : +- CometBroadcastHashJoin + : : : : : : :- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : : : : +- SubqueryBroadcast + : : : : : : : +- BroadcastExchange + : : : : : : : +- CometNativeColumnarToRow + : : : : : : : +- CometProject + : : : : : : : +- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : +- CometBroadcastExchange + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.customer_demographics + : : : : : +- CometBroadcastExchange + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.customer + : : : : +- CometBroadcastExchange + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.customer_demographics + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.customer_address + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.item + +- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometProject + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : : : +- SubqueryBroadcast + : : : : : : +- BroadcastExchange + : : : : : : +- CometNativeColumnarToRow + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- CometBroadcastExchange + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.customer_demographics + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.customer + : : : +- CometBroadcastExchange + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.customer_demographics + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.customer_address + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.date_dim + +- CometBroadcastExchange + +- CometFilter + +- CometNativeScan parquet spark_catalog.default.item -Comet accelerated 92 out of 210 eligible operators (43%). Final plan contains 41 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 216 out of 226 eligible operators (95%). Final plan contains 6 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q20.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q20.native_datafusion/extended.txt index 6634edcbed..c4f54f4289 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q20.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q20.native_datafusion/extended.txt @@ -3,33 +3,29 @@ TakeOrderedAndProject +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] +- CometNativeColumnarToRow +- CometSort - +- CometColumnarExchange - +- HashAggregate - +- CometNativeColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- Filter - : : +- ColumnarToRow - : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- CometNativeColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.item - +- BroadcastExchange - +- CometNativeColumnarToRow - +- CometProject - +- CometFilter - +- CometNativeScan parquet spark_catalog.default.date_dim + +- CometExchange + +- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : +- SubqueryBroadcast + : : +- BroadcastExchange + : : +- CometNativeColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.item + +- CometBroadcastExchange + +- CometProject + +- CometFilter + +- CometNativeScan parquet spark_catalog.default.date_dim -Comet accelerated 12 out of 27 eligible operators (44%). Final plan contains 6 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 22 out of 27 eligible operators (81%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q22.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q22.native_datafusion/extended.txt index 108e3c13f9..a8588fd031 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q22.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q22.native_datafusion/extended.txt @@ -5,27 +5,25 @@ TakeOrderedAndProject +- HashAggregate +- Expand +- Project - +- BroadcastNestedLoopJoin - :- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Filter - : : : +- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.inventory [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- CometNativeColumnarToRow + +- BroadcastNestedLoopJoin [COMET: BroadcastNestedLoopJoin is not supported] + :- CometNativeColumnarToRow + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.inventory + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometNativeColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange : +- CometProject : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.item @@ -33,4 +31,4 @@ TakeOrderedAndProject +- CometNativeColumnarToRow +- CometNativeScan parquet spark_catalog.default.warehouse -Comet accelerated 11 out of 28 eligible operators (39%). Final plan contains 6 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 19 out of 28 eligible operators (67%). Final plan contains 4 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q22a.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q22a.native_datafusion/extended.txt index eaa5438d91..5bd5daea84 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q22a.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q22a.native_datafusion/extended.txt @@ -1,187 +1,159 @@ -TakeOrderedAndProject -+- Union - :- HashAggregate - : +- HashAggregate - : +- HashAggregate - : +- CometNativeColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- Filter - : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.inventory [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.item - : +- BroadcastExchange - : +- CometNativeColumnarToRow - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.warehouse - :- HashAggregate - : +- CometNativeColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- HashAggregate - : +- CometNativeColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- Filter - : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.inventory [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.item - : +- BroadcastExchange - : +- CometNativeColumnarToRow - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.warehouse - :- HashAggregate - : +- CometNativeColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- HashAggregate - : +- CometNativeColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- Filter - : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.inventory [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.item - : +- BroadcastExchange - : +- CometNativeColumnarToRow - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.warehouse - :- HashAggregate - : +- CometNativeColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- HashAggregate - : +- CometNativeColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- Filter - : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.inventory [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.item - : +- BroadcastExchange - : +- CometNativeColumnarToRow - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.warehouse - +- HashAggregate - +- CometNativeColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- HashAggregate - +- CometNativeColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Filter - : : : +- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.inventory [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- CometNativeColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.item - +- BroadcastExchange - +- CometNativeColumnarToRow - +- CometFilter - +- CometNativeScan parquet spark_catalog.default.warehouse +CometNativeColumnarToRow ++- CometTakeOrderedAndProject + +- CometUnion + :- CometHashAggregate + : +- CometHashAggregate + : +- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.inventory + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometNativeColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.item + : +- CometBroadcastExchange + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.warehouse + :- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.inventory + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometNativeColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.item + : +- CometBroadcastExchange + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.warehouse + :- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.inventory + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometNativeColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.item + : +- CometBroadcastExchange + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.warehouse + :- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.inventory + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometNativeColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.item + : +- CometBroadcastExchange + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.warehouse + +- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.inventory + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometNativeColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.item + +- CometBroadcastExchange + +- CometFilter + +- CometNativeScan parquet spark_catalog.default.warehouse -Comet accelerated 64 out of 151 eligible operators (42%). Final plan contains 34 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 141 out of 151 eligible operators (93%). Final plan contains 6 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q27a.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q27a.native_datafusion/extended.txt index 4eec8b0953..5dc06ab86f 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q27a.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q27a.native_datafusion/extended.txt @@ -1,116 +1,109 @@ -TakeOrderedAndProject -+- Union - :- HashAggregate - : +- CometNativeColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- Project - : : : : +- BroadcastHashJoin - : : : : :- Filter - : : : : : +- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.customer_demographics - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.store - : +- BroadcastExchange - : +- CometNativeColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.item - :- HashAggregate - : +- CometNativeColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- Project - : : : : +- BroadcastHashJoin - : : : : :- Filter - : : : : : +- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : +- ReusedSubquery - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.customer_demographics - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.store - : +- BroadcastExchange - : +- CometNativeColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.item - +- HashAggregate - +- CometNativeColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- Filter - : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- ReusedSubquery - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.customer_demographics - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- CometNativeColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.store - +- BroadcastExchange - +- CometNativeColumnarToRow - +- CometFilter - +- CometNativeScan parquet spark_catalog.default.item +CometNativeColumnarToRow ++- CometTakeOrderedAndProject + +- CometUnion + :- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometNativeColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.customer_demographics + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.store + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.item + :- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometNativeColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.customer_demographics + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.store + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.item + +- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometNativeColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.customer_demographics + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.store + +- CometBroadcastExchange + +- CometFilter + +- CometNativeScan parquet spark_catalog.default.item -Comet accelerated 41 out of 95 eligible operators (43%). Final plan contains 19 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 97 out of 103 eligible operators (94%). Final plan contains 4 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q34.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q34.native_datafusion/extended.txt index 40230582bb..1b61be2937 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q34.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q34.native_datafusion/extended.txt @@ -1,47 +1,41 @@ CometNativeColumnarToRow +- CometSort - +- CometColumnarExchange - +- Project - +- BroadcastHashJoin - :- Filter - : +- HashAggregate - : +- CometNativeColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- Filter - : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.store - : +- BroadcastExchange - : +- CometNativeColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.household_demographics - +- BroadcastExchange - +- CometNativeColumnarToRow - +- CometProject - +- CometFilter - +- CometNativeScan parquet spark_catalog.default.customer + +- CometExchange + +- CometProject + +- CometBroadcastHashJoin + :- CometFilter + : +- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometNativeColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.store + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.household_demographics + +- CometBroadcastExchange + +- CometProject + +- CometFilter + +- CometNativeScan parquet spark_catalog.default.customer -Comet accelerated 18 out of 37 eligible operators (48%). Final plan contains 8 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 35 out of 37 eligible operators (94%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q35.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q35.native_datafusion/extended.txt index 40a8ef4a6e..2785c3d0b1 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q35.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q35.native_datafusion/extended.txt @@ -10,46 +10,52 @@ TakeOrderedAndProject : :- Project : : +- Filter : : +- BroadcastHashJoin - : : :- BroadcastHashJoin - : : : :- BroadcastHashJoin - : : : : :- CometNativeColumnarToRow - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.customer - : : : : +- BroadcastExchange - : : : : +- Project - : : : : +- BroadcastHashJoin - : : : : :- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow + : : :- BroadcastHashJoin [COMET: Unsupported join type ExistenceJoin(exists#1)] + : : : :- CometNativeColumnarToRow + : : : : +- CometBroadcastHashJoin + : : : : :- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.customer + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometNativeScan parquet spark_catalog.default.store_sales + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometNativeColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange : : : : +- CometProject : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- BroadcastExchange - : : : +- Project - : : : +- BroadcastHashJoin - : : : :- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- ReusedSubquery - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow + : : : +- CometNativeColumnarToRow + : : : +- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometNativeScan parquet spark_catalog.default.web_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometNativeColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- BroadcastExchange - : : +- Project - : : +- BroadcastHashJoin - : : :- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : +- ReusedSubquery - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow + : : +- CometNativeColumnarToRow + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometNativeColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange : : +- CometProject : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -64,4 +70,4 @@ TakeOrderedAndProject +- CometFilter +- CometNativeScan parquet spark_catalog.default.customer_demographics -Comet accelerated 21 out of 54 eligible operators (38%). Final plan contains 11 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 41 out of 62 eligible operators (66%). Final plan contains 9 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q35a.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q35a.native_datafusion/extended.txt index 9091536e96..cba125f412 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q35a.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q35a.native_datafusion/extended.txt @@ -1,65 +1,66 @@ -TakeOrderedAndProject -+- HashAggregate - +- CometNativeColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- BroadcastHashJoin - : : : :- CometNativeColumnarToRow - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.customer - : : : +- BroadcastExchange - : : : +- Project - : : : +- BroadcastHashJoin - : : : :- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- Union - : : :- Project - : : : +- BroadcastHashJoin - : : : :- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- ReusedSubquery - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- Project - : : +- BroadcastHashJoin - : : :- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : +- ReusedSubquery - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- CometNativeColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.customer_address - +- BroadcastExchange - +- CometNativeColumnarToRow - +- CometProject - +- CometFilter - +- CometNativeScan parquet spark_catalog.default.customer_demographics +CometNativeColumnarToRow ++- CometTakeOrderedAndProject + +- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.customer + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometNativeScan parquet spark_catalog.default.store_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometNativeColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometUnion + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometNativeScan parquet spark_catalog.default.web_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometNativeColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometNativeColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.customer_address + +- CometBroadcastExchange + +- CometProject + +- CometFilter + +- CometNativeScan parquet spark_catalog.default.customer_demographics -Comet accelerated 21 out of 52 eligible operators (40%). Final plan contains 11 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 54 out of 60 eligible operators (90%). Final plan contains 4 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q36a.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q36a.native_datafusion/extended.txt index 1fc86dc5a8..0a69bca968 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q36a.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q36a.native_datafusion/extended.txt @@ -5,118 +5,101 @@ TakeOrderedAndProject +- CometSort +- CometExchange +- CometHashAggregate - +- CometColumnarExchange - +- HashAggregate - +- Union - :- HashAggregate - : +- CometNativeColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- Filter - : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.item - : +- BroadcastExchange - : +- CometNativeColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.store - :- HashAggregate - : +- CometNativeColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- HashAggregate - : +- CometNativeColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- Filter - : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.item - : +- BroadcastExchange - : +- CometNativeColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.store - +- HashAggregate - +- CometNativeColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- HashAggregate - +- CometNativeColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Filter - : : : +- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- CometNativeColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.item - +- BroadcastExchange - +- CometNativeColumnarToRow - +- CometProject - +- CometFilter - +- CometNativeScan parquet spark_catalog.default.store + +- CometExchange + +- CometHashAggregate + +- CometUnion + :- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometNativeColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.item + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.store + :- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometNativeColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.item + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.store + +- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometNativeColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.item + +- CometBroadcastExchange + +- CometProject + +- CometFilter + +- CometNativeScan parquet spark_catalog.default.store -Comet accelerated 45 out of 99 eligible operators (45%). Final plan contains 21 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 90 out of 99 eligible operators (90%). Final plan contains 4 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q47.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q47.native_datafusion/extended.txt index ab44bbef5c..0886059452 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q47.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q47.native_datafusion/extended.txt @@ -10,111 +10,96 @@ TakeOrderedAndProject : : +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] : : +- CometNativeColumnarToRow : : +- CometSort - : : +- CometColumnarExchange - : : +- HashAggregate - : : +- CometNativeColumnarToRow - : : +- CometColumnarExchange - : : +- HashAggregate - : : +- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- Project - : : : : +- BroadcastHashJoin - : : : : :- CometNativeColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : +- BroadcastExchange - : : : : +- Filter - : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.store + : : +- CometExchange + : : +- CometHashAggregate + : : +- CometExchange + : : +- CometHashAggregate + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : : +- CometBroadcastExchange + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometNativeColumnarToRow + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.store : +- BroadcastExchange : +- Project : +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] : +- CometNativeColumnarToRow : +- CometSort - : +- CometColumnarExchange - : +- HashAggregate - : +- CometNativeColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : +- BroadcastExchange - : : : +- Filter - : : : +- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- CometNativeColumnarToRow - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.store + : +- CometExchange + : +- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : +- CometBroadcastExchange + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometNativeColumnarToRow + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.store +- BroadcastExchange +- Project +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] +- CometNativeColumnarToRow +- CometSort - +- CometColumnarExchange - +- HashAggregate - +- CometNativeColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.item - : : +- BroadcastExchange - : : +- Filter - : : +- ColumnarToRow - : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- CometNativeColumnarToRow - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.date_dim - +- BroadcastExchange - +- CometNativeColumnarToRow - +- CometFilter - +- CometNativeScan parquet spark_catalog.default.store + +- CometExchange + +- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.item + : : +- CometBroadcastExchange + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : +- SubqueryBroadcast + : : +- BroadcastExchange + : : +- CometNativeColumnarToRow + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.date_dim + +- CometBroadcastExchange + +- CometFilter + +- CometNativeScan parquet spark_catalog.default.store -Comet accelerated 36 out of 97 eligible operators (37%). Final plan contains 21 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 75 out of 97 eligible operators (77%). Final plan contains 6 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q49.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q49.native_datafusion/extended.txt index 0090739d55..6d7dadcde9 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q49.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q49.native_datafusion/extended.txt @@ -11,35 +11,31 @@ CometNativeColumnarToRow : +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] : +- CometNativeColumnarToRow : +- CometSort - : +- CometColumnarExchange - : +- HashAggregate - : +- CometNativeColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- BroadcastExchange - : : : +- Project - : : : +- Filter - : : : +- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- CometNativeColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.web_returns - : +- BroadcastExchange - : +- CometNativeColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometExchange + : +- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.web_sales + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometNativeColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.web_returns + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.date_dim :- Project : +- Filter : +- Window @@ -47,30 +43,31 @@ CometNativeColumnarToRow : +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] : +- CometNativeColumnarToRow : +- CometSort - : +- CometColumnarExchange - : +- HashAggregate - : +- CometNativeColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- BroadcastExchange - : : : +- Project - : : : +- Filter - : : : +- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : +- ReusedSubquery - : : +- CometNativeColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.catalog_returns - : +- BroadcastExchange - : +- CometNativeColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometExchange + : +- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometNativeColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.catalog_returns + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.date_dim +- Project +- Filter +- Window @@ -78,29 +75,30 @@ CometNativeColumnarToRow +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] +- CometNativeColumnarToRow +- CometSort - +- CometColumnarExchange - +- HashAggregate - +- CometNativeColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- BroadcastExchange - : : +- Project - : : +- Filter - : : +- ColumnarToRow - : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : +- ReusedSubquery - : +- CometNativeColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.store_returns - +- BroadcastExchange - +- CometNativeColumnarToRow - +- CometProject - +- CometFilter - +- CometNativeScan parquet spark_catalog.default.date_dim + +- CometExchange + +- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : +- SubqueryBroadcast + : : +- BroadcastExchange + : : +- CometNativeColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.store_returns + +- CometBroadcastExchange + +- CometProject + +- CometFilter + +- CometNativeScan parquet spark_catalog.default.date_dim -Comet accelerated 33 out of 87 eligible operators (37%). Final plan contains 17 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 72 out of 95 eligible operators (75%). Final plan contains 7 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q51a.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q51a.native_datafusion/extended.txt index 92ee33eda6..d9b9186c1e 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q51a.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q51a.native_datafusion/extended.txt @@ -23,53 +23,47 @@ TakeOrderedAndProject : : : +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] : : : +- CometNativeColumnarToRow : : : +- CometSort - : : : +- CometColumnarExchange - : : : +- HashAggregate - : : : +- CometNativeColumnarToRow - : : : +- CometColumnarExchange - : : : +- HashAggregate - : : : +- Project - : : : +- BroadcastHashJoin - : : : :- Filter - : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometExchange + : : : +- CometHashAggregate + : : : +- CometExchange + : : : +- CometHashAggregate + : : : +- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.web_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometNativeColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- BroadcastExchange : : +- Project : : +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] : : +- CometNativeColumnarToRow : : +- CometSort - : : +- CometColumnarExchange - : : +- HashAggregate - : : +- CometNativeColumnarToRow - : : +- CometColumnarExchange - : : +- HashAggregate - : : +- Project - : : +- BroadcastHashJoin - : : :- Filter - : : : +- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometExchange + : : +- CometHashAggregate + : : +- CometExchange + : : +- CometHashAggregate + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.web_sales + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometNativeColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim : +- CometSort : +- CometColumnarExchange : +- HashAggregate @@ -82,43 +76,47 @@ TakeOrderedAndProject : : +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] : : +- CometNativeColumnarToRow : : +- CometSort - : : +- CometColumnarExchange - : : +- HashAggregate - : : +- CometNativeColumnarToRow - : : +- CometColumnarExchange - : : +- HashAggregate - : : +- Project - : : +- BroadcastHashJoin - : : :- Filter - : : : +- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : +- ReusedSubquery - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometExchange + : : +- CometHashAggregate + : : +- CometExchange + : : +- CometHashAggregate + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometNativeColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim : +- BroadcastExchange : +- Project : +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] : +- CometNativeColumnarToRow : +- CometSort - : +- CometColumnarExchange - : +- HashAggregate - : +- CometNativeColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- Filter - : : +- ColumnarToRow - : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : +- ReusedSubquery - : +- BroadcastExchange - : +- CometNativeColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometExchange + : +- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : +- SubqueryBroadcast + : : +- BroadcastExchange + : : +- CometNativeColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.date_dim +- BroadcastExchange +- Project +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] @@ -140,53 +138,47 @@ TakeOrderedAndProject : : +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] : : +- CometNativeColumnarToRow : : +- CometSort - : : +- CometColumnarExchange - : : +- HashAggregate - : : +- CometNativeColumnarToRow - : : +- CometColumnarExchange - : : +- HashAggregate - : : +- Project - : : +- BroadcastHashJoin - : : :- Filter - : : : +- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometExchange + : : +- CometHashAggregate + : : +- CometExchange + : : +- CometHashAggregate + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.web_sales + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometNativeColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim : +- BroadcastExchange : +- Project : +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] : +- CometNativeColumnarToRow : +- CometSort - : +- CometColumnarExchange - : +- HashAggregate - : +- CometNativeColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- Filter - : : +- ColumnarToRow - : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- CometNativeColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometExchange + : +- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.web_sales + : : +- SubqueryBroadcast + : : +- BroadcastExchange + : : +- CometNativeColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.date_dim +- CometSort +- CometColumnarExchange +- HashAggregate @@ -199,42 +191,46 @@ TakeOrderedAndProject : +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] : +- CometNativeColumnarToRow : +- CometSort - : +- CometColumnarExchange - : +- HashAggregate - : +- CometNativeColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- Filter - : : +- ColumnarToRow - : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : +- ReusedSubquery - : +- BroadcastExchange - : +- CometNativeColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometExchange + : +- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : +- SubqueryBroadcast + : : +- BroadcastExchange + : : +- CometNativeColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.date_dim +- BroadcastExchange +- Project +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] +- CometNativeColumnarToRow +- CometSort - +- CometColumnarExchange - +- HashAggregate - +- CometNativeColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Project - +- BroadcastHashJoin - :- Filter - : +- ColumnarToRow - : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : +- ReusedSubquery - +- BroadcastExchange - +- CometNativeColumnarToRow - +- CometProject - +- CometFilter - +- CometNativeScan parquet spark_catalog.default.date_dim + +- CometExchange + +- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometProject + +- CometBroadcastHashJoin + :- CometFilter + : +- CometNativeScan parquet spark_catalog.default.store_sales + : +- SubqueryBroadcast + : +- BroadcastExchange + : +- CometNativeColumnarToRow + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.date_dim + +- CometBroadcastExchange + +- CometProject + +- CometFilter + +- CometNativeScan parquet spark_catalog.default.date_dim -Comet accelerated 82 out of 196 eligible operators (41%). Final plan contains 42 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 150 out of 212 eligible operators (70%). Final plan contains 22 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q57.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q57.native_datafusion/extended.txt index 66c510aaa0..024bfeeeea 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q57.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q57.native_datafusion/extended.txt @@ -10,111 +10,96 @@ TakeOrderedAndProject : : +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] : : +- CometNativeColumnarToRow : : +- CometSort - : : +- CometColumnarExchange - : : +- HashAggregate - : : +- CometNativeColumnarToRow - : : +- CometColumnarExchange - : : +- HashAggregate - : : +- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- Project - : : : : +- BroadcastHashJoin - : : : : :- CometNativeColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : +- BroadcastExchange - : : : : +- Filter - : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.call_center + : : +- CometExchange + : : +- CometHashAggregate + : : +- CometExchange + : : +- CometHashAggregate + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : : +- CometBroadcastExchange + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometNativeColumnarToRow + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.call_center : +- BroadcastExchange : +- Project : +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] : +- CometNativeColumnarToRow : +- CometSort - : +- CometColumnarExchange - : +- HashAggregate - : +- CometNativeColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : +- BroadcastExchange - : : : +- Filter - : : : +- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- CometNativeColumnarToRow - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.call_center + : +- CometExchange + : +- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : +- CometBroadcastExchange + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometNativeColumnarToRow + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.call_center +- BroadcastExchange +- Project +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] +- CometNativeColumnarToRow +- CometSort - +- CometColumnarExchange - +- HashAggregate - +- CometNativeColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.item - : : +- BroadcastExchange - : : +- Filter - : : +- ColumnarToRow - : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- CometNativeColumnarToRow - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.date_dim - +- BroadcastExchange - +- CometNativeColumnarToRow - +- CometFilter - +- CometNativeScan parquet spark_catalog.default.call_center + +- CometExchange + +- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.item + : : +- CometBroadcastExchange + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : +- SubqueryBroadcast + : : +- BroadcastExchange + : : +- CometNativeColumnarToRow + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.date_dim + +- CometBroadcastExchange + +- CometFilter + +- CometNativeScan parquet spark_catalog.default.call_center -Comet accelerated 36 out of 97 eligible operators (37%). Final plan contains 21 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 75 out of 97 eligible operators (77%). Final plan contains 6 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q5a.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q5a.native_datafusion/extended.txt index ec2eef025a..9cc06fb361 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q5a.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q5a.native_datafusion/extended.txt @@ -1,322 +1,344 @@ CometNativeColumnarToRow +- CometTakeOrderedAndProject +- CometHashAggregate - +- CometColumnarExchange - +- HashAggregate - +- Union - :- HashAggregate - : +- CometNativeColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Union - : :- HashAggregate - : : +- CometNativeColumnarToRow - : : +- CometColumnarExchange - : : +- HashAggregate - : : +- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- Union - : : : : :- Project - : : : : : +- Filter - : : : : : +- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : +- Project - : : : : +- Filter - : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.store_returns [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- ReusedSubquery - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.store - : :- HashAggregate - : : +- CometNativeColumnarToRow - : : +- CometColumnarExchange - : : +- HashAggregate - : : +- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- Union - : : : : :- Project - : : : : : +- Filter - : : : : : +- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : +- ReusedSubquery - : : : : +- Project - : : : : +- Filter - : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.catalog_returns [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- ReusedSubquery - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.catalog_page - : +- HashAggregate - : +- CometNativeColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Union - : : : :- Project - : : : : +- Filter - : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- ReusedSubquery - : : : +- Project - : : : +- BroadcastHashJoin - : : : :- BroadcastExchange - : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.web_returns [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- ReusedSubquery - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- CometNativeColumnarToRow + +- CometExchange + +- CometHashAggregate + +- CometUnion + :- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometUnion + : :- CometHashAggregate + : : +- CometExchange + : : +- CometHashAggregate + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometUnion + : : : : :- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometNativeColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.store_returns + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometNativeColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.store + : :- CometHashAggregate + : : +- CometExchange + : : +- CometHashAggregate + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometUnion + : : : : :- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometNativeColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.catalog_returns + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometNativeColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.catalog_page + : +- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometUnion + : : : :- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.web_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometNativeColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometBroadcastExchange + : : : : +- CometNativeScan parquet spark_catalog.default.web_returns + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometNativeColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.web_sales + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.web_site + :- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometUnion + : :- CometHashAggregate + : : +- CometExchange + : : +- CometHashAggregate + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometUnion + : : : : :- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometNativeColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.store_returns + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometNativeColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.store + : :- CometHashAggregate + : : +- CometExchange + : : +- CometHashAggregate + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometUnion + : : : : :- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometNativeColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.catalog_returns + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometNativeColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.catalog_page + : +- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometUnion + : : : :- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.web_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometNativeColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometBroadcastExchange + : : : : +- CometNativeScan parquet spark_catalog.default.web_returns + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometNativeColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.web_sales + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange : +- CometProject : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.web_site - :- HashAggregate - : +- CometNativeColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- HashAggregate - : +- CometNativeColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Union - : :- HashAggregate - : : +- CometNativeColumnarToRow - : : +- CometColumnarExchange - : : +- HashAggregate - : : +- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- Union - : : : : :- Project - : : : : : +- Filter - : : : : : +- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : +- Project - : : : : +- Filter - : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.store_returns [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- ReusedSubquery - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.store - : :- HashAggregate - : : +- CometNativeColumnarToRow - : : +- CometColumnarExchange - : : +- HashAggregate - : : +- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- Union - : : : : :- Project - : : : : : +- Filter - : : : : : +- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : +- ReusedSubquery - : : : : +- Project - : : : : +- Filter - : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.catalog_returns [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- ReusedSubquery - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.catalog_page - : +- HashAggregate - : +- CometNativeColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Union - : : : :- Project - : : : : +- Filter - : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- ReusedSubquery - : : : +- Project - : : : +- BroadcastHashJoin - : : : :- BroadcastExchange - : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.web_returns [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- ReusedSubquery - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- CometNativeColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.web_site - +- HashAggregate - +- CometNativeColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- HashAggregate - +- CometNativeColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Union - :- HashAggregate - : +- CometNativeColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Union - : : : :- Project - : : : : +- Filter - : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- Project - : : : +- Filter - : : : +- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.store_returns [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : +- ReusedSubquery - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- CometNativeColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.store - :- HashAggregate - : +- CometNativeColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Union - : : : :- Project - : : : : +- Filter - : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- ReusedSubquery - : : : +- Project - : : : +- Filter - : : : +- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.catalog_returns [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : +- ReusedSubquery - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- CometNativeColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.catalog_page - +- HashAggregate - +- CometNativeColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- Union - : : :- Project - : : : +- Filter - : : : +- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : +- ReusedSubquery - : : +- Project - : : +- BroadcastHashJoin - : : :- BroadcastExchange - : : : +- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.web_returns [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : +- ReusedSubquery - : : +- CometNativeColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.web_sales - : +- BroadcastExchange - : +- CometNativeColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.date_dim - +- BroadcastExchange - +- CometNativeColumnarToRow - +- CometProject - +- CometFilter - +- CometNativeScan parquet spark_catalog.default.web_site + +- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometUnion + :- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometUnion + : : : :- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometNativeColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.store_returns + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometNativeColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.store + :- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometUnion + : : : :- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometNativeColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.catalog_returns + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometNativeColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.catalog_page + +- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometUnion + : : :- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.web_sales + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometNativeColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometBroadcastExchange + : : : +- CometNativeScan parquet spark_catalog.default.web_returns + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometNativeColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.web_sales + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.date_dim + +- CometBroadcastExchange + +- CometProject + +- CometFilter + +- CometNativeScan parquet spark_catalog.default.web_site -Comet accelerated 89 out of 263 eligible operators (33%). Final plan contains 57 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 287 out of 323 eligible operators (88%). Final plan contains 19 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q6.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q6.native_datafusion/extended.txt index f2cdb50e03..1cb21cb650 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q6.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q6.native_datafusion/extended.txt @@ -1,68 +1,57 @@ -TakeOrderedAndProject -+- Filter - +- HashAggregate - +- CometNativeColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.customer_address - : : : +- CometBroadcastExchange - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.customer - : : +- BroadcastExchange - : : +- Filter - : : +- ColumnarToRow - : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometProject - : : +- CometFilter - : : : +- Subquery - : : : +- CometNativeColumnarToRow - : : : +- CometHashAggregate - : : : +- CometExchange - : : : +- CometHashAggregate - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- CometNativeColumnarToRow - : +- CometProject - : +- CometFilter - : : +- Subquery - : : +- CometNativeColumnarToRow - : : +- CometHashAggregate - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- CometNativeScan parquet spark_catalog.default.date_dim - +- BroadcastExchange - +- CometNativeColumnarToRow - +- CometProject - +- CometBroadcastHashJoin - :- CometFilter - : +- CometNativeScan parquet spark_catalog.default.item - +- CometBroadcastExchange - +- CometFilter - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometFilter - +- CometNativeScan parquet spark_catalog.default.item +CometNativeColumnarToRow ++- CometTakeOrderedAndProject + +- CometFilter + +- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.customer_address + : : : +- CometBroadcastExchange + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.customer + : : +- CometBroadcastExchange + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : +- SubqueryBroadcast + : : +- BroadcastExchange + : : +- CometNativeColumnarToRow + : : +- CometProject + : : +- CometFilter + : : : +- Subquery + : : : +- CometNativeColumnarToRow + : : : +- CometHashAggregate + : : : +- CometExchange + : : : +- CometHashAggregate + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : : +- ReusedSubquery + : +- CometNativeScan parquet spark_catalog.default.date_dim + +- CometBroadcastExchange + +- CometProject + +- CometBroadcastHashJoin + :- CometFilter + : +- CometNativeScan parquet spark_catalog.default.item + +- CometBroadcastExchange + +- CometFilter + +- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometProject + +- CometFilter + +- CometNativeScan parquet spark_catalog.default.item -Comet accelerated 39 out of 58 eligible operators (67%). Final plan contains 8 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 48 out of 52 eligible operators (92%). Final plan contains 3 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q64.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q64.native_datafusion/extended.txt index 98db64e5b0..f1fe6fb3e3 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q64.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q64.native_datafusion/extended.txt @@ -40,22 +40,20 @@ CometNativeColumnarToRow : : : : : : : : : : : : : : : :- CometProject : : : : : : : : : : : : : : : : +- CometSortMergeJoin : : : : : : : : : : : : : : : : :- CometSort - : : : : : : : : : : : : : : : : : +- CometColumnarExchange - : : : : : : : : : : : : : : : : : +- Project - : : : : : : : : : : : : : : : : : +- BroadcastHashJoin - : : : : : : : : : : : : : : : : : :- BroadcastExchange - : : : : : : : : : : : : : : : : : : +- Filter - : : : : : : : : : : : : : : : : : : +- ColumnarToRow - : : : : : : : : : : : : : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : : : : : : : : : : : : : : +- SubqueryBroadcast - : : : : : : : : : : : : : : : : : : +- BroadcastExchange - : : : : : : : : : : : : : : : : : : +- CometNativeColumnarToRow - : : : : : : : : : : : : : : : : : : +- CometFilter - : : : : : : : : : : : : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : : : : : : : : : : : : : : +- CometNativeColumnarToRow - : : : : : : : : : : : : : : : : : +- CometProject - : : : : : : : : : : : : : : : : : +- CometFilter - : : : : : : : : : : : : : : : : : +- CometNativeScan parquet spark_catalog.default.store_returns + : : : : : : : : : : : : : : : : : +- CometExchange + : : : : : : : : : : : : : : : : : +- CometProject + : : : : : : : : : : : : : : : : : +- CometBroadcastHashJoin + : : : : : : : : : : : : : : : : : :- CometBroadcastExchange + : : : : : : : : : : : : : : : : : : +- CometFilter + : : : : : : : : : : : : : : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : : : : : : : : : : : : : : : +- SubqueryBroadcast + : : : : : : : : : : : : : : : : : : +- BroadcastExchange + : : : : : : : : : : : : : : : : : : +- CometNativeColumnarToRow + : : : : : : : : : : : : : : : : : : +- CometFilter + : : : : : : : : : : : : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : : : : : : : : : : : : +- CometProject + : : : : : : : : : : : : : : : : : +- CometFilter + : : : : : : : : : : : : : : : : : +- CometNativeScan parquet spark_catalog.default.store_returns : : : : : : : : : : : : : : : : +- CometSort : : : : : : : : : : : : : : : : +- CometProject : : : : : : : : : : : : : : : : +- CometFilter @@ -162,22 +160,20 @@ CometNativeColumnarToRow : : : : : : : : : : : : : : :- CometProject : : : : : : : : : : : : : : : +- CometSortMergeJoin : : : : : : : : : : : : : : : :- CometSort - : : : : : : : : : : : : : : : : +- CometColumnarExchange - : : : : : : : : : : : : : : : : +- Project - : : : : : : : : : : : : : : : : +- BroadcastHashJoin - : : : : : : : : : : : : : : : : :- BroadcastExchange - : : : : : : : : : : : : : : : : : +- Filter - : : : : : : : : : : : : : : : : : +- ColumnarToRow - : : : : : : : : : : : : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : : : : : : : : : : : : : +- SubqueryBroadcast - : : : : : : : : : : : : : : : : : +- BroadcastExchange - : : : : : : : : : : : : : : : : : +- CometNativeColumnarToRow - : : : : : : : : : : : : : : : : : +- CometFilter - : : : : : : : : : : : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : : : : : : : : : : : : : +- CometNativeColumnarToRow - : : : : : : : : : : : : : : : : +- CometProject - : : : : : : : : : : : : : : : : +- CometFilter - : : : : : : : : : : : : : : : : +- CometNativeScan parquet spark_catalog.default.store_returns + : : : : : : : : : : : : : : : : +- CometExchange + : : : : : : : : : : : : : : : : +- CometProject + : : : : : : : : : : : : : : : : +- CometBroadcastHashJoin + : : : : : : : : : : : : : : : : :- CometBroadcastExchange + : : : : : : : : : : : : : : : : : +- CometFilter + : : : : : : : : : : : : : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : : : : : : : : : : : : : : +- SubqueryBroadcast + : : : : : : : : : : : : : : : : : +- BroadcastExchange + : : : : : : : : : : : : : : : : : +- CometNativeColumnarToRow + : : : : : : : : : : : : : : : : : +- CometFilter + : : : : : : : : : : : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : : : : : : : : : : : +- CometProject + : : : : : : : : : : : : : : : : +- CometFilter + : : : : : : : : : : : : : : : : +- CometNativeScan parquet spark_catalog.default.store_returns : : : : : : : : : : : : : : : +- CometSort : : : : : : : : : : : : : : : +- CometProject : : : : : : : : : : : : : : : +- CometFilter @@ -248,4 +244,4 @@ CometNativeColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.item -Comet accelerated 228 out of 242 eligible operators (94%). Final plan contains 7 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 238 out of 242 eligible operators (98%). Final plan contains 3 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q67a.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q67a.native_datafusion/extended.txt index bf553c1971..abf8dad401 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q67a.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q67a.native_datafusion/extended.txt @@ -5,346 +5,294 @@ TakeOrderedAndProject +- CometNativeColumnarToRow +- CometSort +- CometColumnarExchange - +- WindowGroupLimit - +- Sort - +- Union - :- HashAggregate - : +- CometNativeColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- Filter - : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.store - : +- BroadcastExchange - : +- CometNativeColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.item - :- HashAggregate - : +- CometNativeColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- HashAggregate - : +- CometNativeColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- Filter - : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.store - : +- BroadcastExchange - : +- CometNativeColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.item - :- HashAggregate - : +- CometNativeColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- HashAggregate - : +- CometNativeColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- Filter - : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.store - : +- BroadcastExchange - : +- CometNativeColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.item - :- HashAggregate - : +- CometNativeColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- HashAggregate - : +- CometNativeColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- Filter - : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.store - : +- BroadcastExchange - : +- CometNativeColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.item - :- HashAggregate - : +- CometNativeColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- HashAggregate - : +- CometNativeColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- Filter - : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.store - : +- BroadcastExchange - : +- CometNativeColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.item - :- HashAggregate - : +- CometNativeColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- HashAggregate - : +- CometNativeColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- Filter - : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.store - : +- BroadcastExchange - : +- CometNativeColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.item - :- HashAggregate - : +- CometNativeColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- HashAggregate - : +- CometNativeColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- Filter - : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.store - : +- BroadcastExchange - : +- CometNativeColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.item - :- HashAggregate - : +- CometNativeColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- HashAggregate - : +- CometNativeColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- Filter - : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.store - : +- BroadcastExchange - : +- CometNativeColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.item - +- HashAggregate - +- CometNativeColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- HashAggregate - +- CometNativeColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Filter - : : : +- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- CometNativeColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.store - +- BroadcastExchange - +- CometNativeColumnarToRow - +- CometProject - +- CometFilter - +- CometNativeScan parquet spark_catalog.default.item + +- WindowGroupLimit [COMET: WindowGroupLimit is not supported] + +- CometNativeColumnarToRow + +- CometSort + +- CometUnion + :- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometNativeColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.store + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.item + :- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometNativeColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.store + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.item + :- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometNativeColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.store + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.item + :- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometNativeColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.store + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.item + :- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometNativeColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.store + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.item + :- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometNativeColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.store + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.item + :- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometNativeColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.store + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.item + :- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometNativeColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.store + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.item + +- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometNativeColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.store + +- CometBroadcastExchange + +- CometProject + +- CometFilter + +- CometNativeScan parquet spark_catalog.default.item -Comet accelerated 127 out of 285 eligible operators (44%). Final plan contains 63 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 262 out of 285 eligible operators (91%). Final plan contains 11 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q70a.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q70a.native_datafusion/extended.txt index 3214d035b2..4ba9e6eb00 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q70a.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q70a.native_datafusion/extended.txt @@ -14,19 +14,18 @@ TakeOrderedAndProject : +- HashAggregate : +- Project : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Filter - : : : +- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow + : :- CometNativeColumnarToRow + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometNativeColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange : : +- CometProject : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -40,30 +39,32 @@ TakeOrderedAndProject : +- Project : +- Filter : +- Window - : +- WindowGroupLimit - : +- Sort - : +- HashAggregate - : +- CometNativeColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Filter - : : : +- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : +- ReusedSubquery - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.store - : +- BroadcastExchange - : +- CometNativeColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- WindowGroupLimit [COMET: WindowGroupLimit is not supported] + : +- CometNativeColumnarToRow + : +- CometSort + : +- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometNativeColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.store + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.date_dim :- HashAggregate : +- CometNativeColumnarToRow : +- CometColumnarExchange @@ -74,19 +75,18 @@ TakeOrderedAndProject : +- HashAggregate : +- Project : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Filter - : : : +- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow + : :- CometNativeColumnarToRow + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometNativeColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange : : +- CometProject : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -100,30 +100,32 @@ TakeOrderedAndProject : +- Project : +- Filter : +- Window - : +- WindowGroupLimit - : +- Sort - : +- HashAggregate - : +- CometNativeColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Filter - : : : +- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : +- ReusedSubquery - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.store - : +- BroadcastExchange - : +- CometNativeColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- WindowGroupLimit [COMET: WindowGroupLimit is not supported] + : +- CometNativeColumnarToRow + : +- CometSort + : +- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometNativeColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.store + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.date_dim +- HashAggregate +- CometNativeColumnarToRow +- CometColumnarExchange @@ -134,19 +136,18 @@ TakeOrderedAndProject +- HashAggregate +- Project +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- Filter - : : +- ColumnarToRow - : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- CometNativeColumnarToRow + :- CometNativeColumnarToRow + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : +- SubqueryBroadcast + : : +- BroadcastExchange + : : +- CometNativeColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange : +- CometProject : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -160,29 +161,31 @@ TakeOrderedAndProject +- Project +- Filter +- Window - +- WindowGroupLimit - +- Sort - +- HashAggregate - +- CometNativeColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- Filter - : : +- ColumnarToRow - : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : +- ReusedSubquery - : +- BroadcastExchange - : +- CometNativeColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.store - +- BroadcastExchange - +- CometNativeColumnarToRow - +- CometProject - +- CometFilter - +- CometNativeScan parquet spark_catalog.default.date_dim + +- WindowGroupLimit [COMET: WindowGroupLimit is not supported] + +- CometNativeColumnarToRow + +- CometSort + +- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : +- SubqueryBroadcast + : : +- BroadcastExchange + : : +- CometNativeColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.store + +- CometBroadcastExchange + +- CometProject + +- CometFilter + +- CometNativeScan parquet spark_catalog.default.date_dim -Comet accelerated 54 out of 156 eligible operators (34%). Final plan contains 30 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 111 out of 168 eligible operators (66%). Final plan contains 21 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q72.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q72.native_datafusion/extended.txt index 62c2b17b7e..623a68ddab 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q72.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q72.native_datafusion/extended.txt @@ -6,77 +6,67 @@ CometNativeColumnarToRow +- CometProject +- CometSortMergeJoin :- CometSort - : +- CometColumnarExchange - : +- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- Project - : : : : +- BroadcastHashJoin - : : : : :- Project - : : : : : +- BroadcastHashJoin - : : : : : :- Project - : : : : : : +- BroadcastHashJoin - : : : : : : :- Project - : : : : : : : +- BroadcastHashJoin - : : : : : : : :- Project - : : : : : : : : +- BroadcastHashJoin - : : : : : : : : :- Project - : : : : : : : : : +- BroadcastHashJoin - : : : : : : : : : :- Filter - : : : : : : : : : : +- ColumnarToRow - : : : : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : : : : : : +- SubqueryBroadcast - : : : : : : : : : : +- BroadcastExchange - : : : : : : : : : : +- CometNativeColumnarToRow - : : : : : : : : : : +- CometProject - : : : : : : : : : : +- CometFilter - : : : : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : : : : : : +- BroadcastExchange - : : : : : : : : : +- CometNativeColumnarToRow - : : : : : : : : : +- CometFilter - : : : : : : : : : +- CometNativeScan parquet spark_catalog.default.inventory - : : : : : : : : +- BroadcastExchange - : : : : : : : : +- CometNativeColumnarToRow - : : : : : : : : +- CometFilter - : : : : : : : : +- CometNativeScan parquet spark_catalog.default.warehouse - : : : : : : : +- BroadcastExchange - : : : : : : : +- CometNativeColumnarToRow - : : : : : : : +- CometFilter - : : : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : : : +- BroadcastExchange - : : : : : : +- CometNativeColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.customer_demographics - : : : : : +- BroadcastExchange - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.household_demographics - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- CometNativeColumnarToRow - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.promotion + : +- CometExchange + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometProject + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometProject + : : : : : : +- CometBroadcastHashJoin + : : : : : : :- CometProject + : : : : : : : +- CometBroadcastHashJoin + : : : : : : : :- CometProject + : : : : : : : : +- CometBroadcastHashJoin + : : : : : : : : :- CometProject + : : : : : : : : : +- CometBroadcastHashJoin + : : : : : : : : : :- CometFilter + : : : : : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : : : : : : : +- SubqueryBroadcast + : : : : : : : : : : +- BroadcastExchange + : : : : : : : : : : +- CometNativeColumnarToRow + : : : : : : : : : : +- CometProject + : : : : : : : : : : +- CometFilter + : : : : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : : : : +- CometBroadcastExchange + : : : : : : : : : +- CometFilter + : : : : : : : : : +- CometNativeScan parquet spark_catalog.default.inventory + : : : : : : : : +- CometBroadcastExchange + : : : : : : : : +- CometFilter + : : : : : : : : +- CometNativeScan parquet spark_catalog.default.warehouse + : : : : : : : +- CometBroadcastExchange + : : : : : : : +- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : : : : +- CometBroadcastExchange + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.customer_demographics + : : : : : +- CometBroadcastExchange + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.household_demographics + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.promotion +- CometSort +- CometExchange +- CometProject +- CometFilter +- CometNativeScan parquet spark_catalog.default.catalog_returns -Comet accelerated 37 out of 68 eligible operators (54%). Final plan contains 12 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 66 out of 68 eligible operators (97%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q74.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q74.native_datafusion/extended.txt index 5d218998be..51a3f5ca94 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q74.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q74.native_datafusion/extended.txt @@ -1,105 +1,98 @@ -TakeOrderedAndProject -+- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- BroadcastHashJoin - : : :- Filter - : : : +- HashAggregate - : : : +- CometNativeColumnarToRow - : : : +- CometColumnarExchange - : : : +- HashAggregate - : : : +- Project - : : : +- BroadcastHashJoin - : : : :- Project - : : : : +- BroadcastHashJoin - : : : : :- CometNativeColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.customer - : : : : +- BroadcastExchange - : : : : +- Filter - : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- HashAggregate - : : +- CometNativeColumnarToRow - : : +- CometColumnarExchange - : : +- HashAggregate - : : +- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.customer - : : : +- BroadcastExchange - : : : +- Filter - : : : +- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- Filter - : +- HashAggregate - : +- CometNativeColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.customer - : : +- BroadcastExchange - : : +- Filter - : : +- ColumnarToRow - : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : +- ReusedSubquery - : +- BroadcastExchange - : +- CometNativeColumnarToRow - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.date_dim - +- BroadcastExchange - +- HashAggregate - +- CometNativeColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- CometNativeColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.customer - : +- BroadcastExchange - : +- Filter - : +- ColumnarToRow - : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : +- ReusedSubquery - +- BroadcastExchange - +- CometNativeColumnarToRow - +- CometFilter - +- CometNativeScan parquet spark_catalog.default.date_dim +CometNativeColumnarToRow ++- CometTakeOrderedAndProject + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometHashAggregate + : : : +- CometExchange + : : : +- CometHashAggregate + : : : +- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.customer + : : : : +- CometBroadcastExchange + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometNativeColumnarToRow + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometHashAggregate + : : +- CometExchange + : : +- CometHashAggregate + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.customer + : : : +- CometBroadcastExchange + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometNativeColumnarToRow + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometFilter + : +- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.customer + : : +- CometBroadcastExchange + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.web_sales + : : +- SubqueryBroadcast + : : +- BroadcastExchange + : : +- CometNativeColumnarToRow + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.date_dim + +- CometBroadcastExchange + +- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.customer + : +- CometBroadcastExchange + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.web_sales + : +- SubqueryBroadcast + : +- BroadcastExchange + : +- CometNativeColumnarToRow + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.date_dim + +- CometBroadcastExchange + +- CometFilter + +- CometNativeScan parquet spark_catalog.default.date_dim -Comet accelerated 28 out of 85 eligible operators (32%). Final plan contains 18 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 83 out of 91 eligible operators (91%). Final plan contains 5 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q75.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q75.native_datafusion/extended.txt index 42106a0ffd..a943308f1e 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q75.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q75.native_datafusion/extended.txt @@ -15,28 +15,25 @@ CometNativeColumnarToRow : :- CometProject : : +- CometSortMergeJoin : : :- CometSort - : : : +- CometColumnarExchange - : : : +- Project - : : : +- BroadcastHashJoin - : : : :- Project - : : : : +- BroadcastHashJoin - : : : : :- Filter - : : : : : +- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometExchange + : : : +- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometNativeColumnarToRow + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : +- CometBroadcastExchange + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- CometSort : : +- CometExchange : : +- CometProject @@ -45,24 +42,25 @@ CometNativeColumnarToRow : :- CometProject : : +- CometSortMergeJoin : : :- CometSort - : : : +- CometColumnarExchange - : : : +- Project - : : : +- BroadcastHashJoin - : : : :- Project - : : : : +- BroadcastHashJoin - : : : : :- Filter - : : : : : +- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : +- ReusedSubquery - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometExchange + : : : +- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometNativeColumnarToRow + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : +- CometBroadcastExchange + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- CometSort : : +- CometExchange : : +- CometProject @@ -71,24 +69,25 @@ CometNativeColumnarToRow : +- CometProject : +- CometSortMergeJoin : :- CometSort - : : +- CometColumnarExchange - : : +- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- Filter - : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- ReusedSubquery - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.item - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometExchange + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.web_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometNativeColumnarToRow + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.item + : : +- CometBroadcastExchange + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim : +- CometSort : +- CometExchange : +- CometProject @@ -107,28 +106,25 @@ CometNativeColumnarToRow :- CometProject : +- CometSortMergeJoin : :- CometSort - : : +- CometColumnarExchange - : : +- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- Filter - : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.item - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometExchange + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometNativeColumnarToRow + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.item + : : +- CometBroadcastExchange + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim : +- CometSort : +- CometExchange : +- CometProject @@ -137,24 +133,25 @@ CometNativeColumnarToRow :- CometProject : +- CometSortMergeJoin : :- CometSort - : : +- CometColumnarExchange - : : +- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- Filter - : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- ReusedSubquery - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.item - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometExchange + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometNativeColumnarToRow + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.item + : : +- CometBroadcastExchange + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim : +- CometSort : +- CometExchange : +- CometProject @@ -163,28 +160,29 @@ CometNativeColumnarToRow +- CometProject +- CometSortMergeJoin :- CometSort - : +- CometColumnarExchange - : +- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Filter - : : : +- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : +- ReusedSubquery - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.item - : +- BroadcastExchange - : +- CometNativeColumnarToRow - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometExchange + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.web_sales + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometNativeColumnarToRow + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.item + : +- CometBroadcastExchange + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.date_dim +- CometSort +- CometExchange +- CometProject +- CometFilter +- CometNativeScan parquet spark_catalog.default.web_returns -Comet accelerated 111 out of 167 eligible operators (66%). Final plan contains 21 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 167 out of 179 eligible operators (93%). Final plan contains 7 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q77a.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q77a.native_datafusion/extended.txt index 178b946ecb..24407fc0a2 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q77a.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q77a.native_datafusion/extended.txt @@ -9,132 +9,139 @@ CometNativeColumnarToRow : +- CometColumnarExchange : +- HashAggregate : +- Union + : :- CometNativeColumnarToRow + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometHashAggregate + : : : +- CometExchange + : : : +- CometHashAggregate + : : : +- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometNativeColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.store + : : +- CometBroadcastExchange + : : +- CometHashAggregate + : : +- CometExchange + : : +- CometHashAggregate + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.store_returns + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometNativeColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.store : :- Project - : : +- BroadcastHashJoin - : : :- HashAggregate - : : : +- CometNativeColumnarToRow - : : : +- CometColumnarExchange - : : : +- HashAggregate - : : : +- Project - : : : +- BroadcastHashJoin - : : : :- Project - : : : : +- BroadcastHashJoin - : : : : :- Filter - : : : : : +- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.store - : : +- BroadcastExchange - : : +- HashAggregate - : : +- CometNativeColumnarToRow - : : +- CometColumnarExchange - : : +- HashAggregate - : : +- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- Filter - : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.store_returns [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- ReusedSubquery - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.store - : :- Project - : : +- BroadcastNestedLoopJoin + : : +- BroadcastNestedLoopJoin [COMET: BroadcastNestedLoopJoin is not supported] : : :- BroadcastExchange - : : : +- HashAggregate - : : : +- CometNativeColumnarToRow - : : : +- CometColumnarExchange - : : : +- HashAggregate - : : : +- Project - : : : +- BroadcastHashJoin - : : : :- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- ReusedSubquery - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- HashAggregate - : : +- CometNativeColumnarToRow - : : +- CometColumnarExchange - : : +- HashAggregate - : : +- Project - : : +- BroadcastHashJoin - : : :- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.catalog_returns [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : +- ReusedSubquery - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- Project - : +- BroadcastHashJoin - : :- HashAggregate - : : +- CometNativeColumnarToRow - : : +- CometColumnarExchange - : : +- HashAggregate - : : +- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- Filter - : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- ReusedSubquery - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.web_page - : +- BroadcastExchange - : +- HashAggregate - : +- CometNativeColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Filter - : : : +- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.web_returns [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : +- ReusedSubquery - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- CometNativeColumnarToRow - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.web_page + : : : +- CometNativeColumnarToRow + : : : +- CometHashAggregate + : : : +- CometExchange + : : : +- CometHashAggregate + : : : +- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometNativeColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometNativeColumnarToRow + : : +- CometHashAggregate + : : +- CometExchange + : : +- CometHashAggregate + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometNativeScan parquet spark_catalog.default.catalog_returns + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometNativeColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometNativeColumnarToRow + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometHashAggregate + : : +- CometExchange + : : +- CometHashAggregate + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.web_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometNativeColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.web_page + : +- CometBroadcastExchange + : +- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.web_returns + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometNativeColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.web_page :- HashAggregate : +- CometNativeColumnarToRow : +- CometColumnarExchange @@ -144,132 +151,139 @@ CometNativeColumnarToRow : +- CometColumnarExchange : +- HashAggregate : +- Union + : :- CometNativeColumnarToRow + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometHashAggregate + : : : +- CometExchange + : : : +- CometHashAggregate + : : : +- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometNativeColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.store + : : +- CometBroadcastExchange + : : +- CometHashAggregate + : : +- CometExchange + : : +- CometHashAggregate + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.store_returns + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometNativeColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.store : :- Project - : : +- BroadcastHashJoin - : : :- HashAggregate - : : : +- CometNativeColumnarToRow - : : : +- CometColumnarExchange - : : : +- HashAggregate - : : : +- Project - : : : +- BroadcastHashJoin - : : : :- Project - : : : : +- BroadcastHashJoin - : : : : :- Filter - : : : : : +- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.store - : : +- BroadcastExchange - : : +- HashAggregate - : : +- CometNativeColumnarToRow - : : +- CometColumnarExchange - : : +- HashAggregate - : : +- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- Filter - : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.store_returns [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- ReusedSubquery - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.store - : :- Project - : : +- BroadcastNestedLoopJoin + : : +- BroadcastNestedLoopJoin [COMET: BroadcastNestedLoopJoin is not supported] : : :- BroadcastExchange - : : : +- HashAggregate - : : : +- CometNativeColumnarToRow - : : : +- CometColumnarExchange - : : : +- HashAggregate - : : : +- Project - : : : +- BroadcastHashJoin - : : : :- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- ReusedSubquery - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- HashAggregate - : : +- CometNativeColumnarToRow - : : +- CometColumnarExchange - : : +- HashAggregate - : : +- Project - : : +- BroadcastHashJoin - : : :- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.catalog_returns [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : +- ReusedSubquery - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- Project - : +- BroadcastHashJoin - : :- HashAggregate - : : +- CometNativeColumnarToRow - : : +- CometColumnarExchange - : : +- HashAggregate - : : +- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- Filter - : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- ReusedSubquery - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.web_page - : +- BroadcastExchange - : +- HashAggregate - : +- CometNativeColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Filter - : : : +- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.web_returns [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : +- ReusedSubquery - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- CometNativeColumnarToRow - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.web_page + : : : +- CometNativeColumnarToRow + : : : +- CometHashAggregate + : : : +- CometExchange + : : : +- CometHashAggregate + : : : +- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometNativeColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometNativeColumnarToRow + : : +- CometHashAggregate + : : +- CometExchange + : : +- CometHashAggregate + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometNativeScan parquet spark_catalog.default.catalog_returns + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometNativeColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometNativeColumnarToRow + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometHashAggregate + : : +- CometExchange + : : +- CometHashAggregate + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.web_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometNativeColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.web_page + : +- CometBroadcastExchange + : +- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.web_returns + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometNativeColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.web_page +- HashAggregate +- CometNativeColumnarToRow +- CometColumnarExchange @@ -279,131 +293,138 @@ CometNativeColumnarToRow +- CometColumnarExchange +- HashAggregate +- Union + :- CometNativeColumnarToRow + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometHashAggregate + : : +- CometExchange + : : +- CometHashAggregate + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometNativeColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.store + : +- CometBroadcastExchange + : +- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.store_returns + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometNativeColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.store :- Project - : +- BroadcastHashJoin - : :- HashAggregate - : : +- CometNativeColumnarToRow - : : +- CometColumnarExchange - : : +- HashAggregate - : : +- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- Filter - : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.store - : +- BroadcastExchange - : +- HashAggregate - : +- CometNativeColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Filter - : : : +- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.store_returns [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : +- ReusedSubquery - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- CometNativeColumnarToRow - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.store - :- Project - : +- BroadcastNestedLoopJoin + : +- BroadcastNestedLoopJoin [COMET: BroadcastNestedLoopJoin is not supported] : :- BroadcastExchange - : : +- HashAggregate - : : +- CometNativeColumnarToRow - : : +- CometColumnarExchange - : : +- HashAggregate - : : +- Project - : : +- BroadcastHashJoin - : : :- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : +- ReusedSubquery - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- HashAggregate - : +- CometNativeColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- ColumnarToRow - : : +- Scan parquet spark_catalog.default.catalog_returns [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : +- ReusedSubquery - : +- BroadcastExchange - : +- CometNativeColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.date_dim - +- Project - +- BroadcastHashJoin - :- HashAggregate - : +- CometNativeColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Filter - : : : +- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : +- ReusedSubquery - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- CometNativeColumnarToRow - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.web_page - +- BroadcastExchange - +- HashAggregate - +- CometNativeColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- Filter - : : +- ColumnarToRow - : : +- Scan parquet spark_catalog.default.web_returns [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : +- ReusedSubquery - : +- BroadcastExchange - : +- CometNativeColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.date_dim - +- BroadcastExchange - +- CometNativeColumnarToRow - +- CometFilter - +- CometNativeScan parquet spark_catalog.default.web_page + : : +- CometNativeColumnarToRow + : : +- CometHashAggregate + : : +- CometExchange + : : +- CometHashAggregate + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometNativeColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometNativeColumnarToRow + : +- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometNativeScan parquet spark_catalog.default.catalog_returns + : : +- SubqueryBroadcast + : : +- BroadcastExchange + : : +- CometNativeColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.date_dim + +- CometNativeColumnarToRow + +- CometProject + +- CometBroadcastHashJoin + :- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.web_sales + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometNativeColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.web_page + +- CometBroadcastExchange + +- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.web_returns + : : +- SubqueryBroadcast + : : +- BroadcastExchange + : : +- CometNativeColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.date_dim + +- CometBroadcastExchange + +- CometFilter + +- CometNativeScan parquet spark_catalog.default.web_page -Comet accelerated 113 out of 332 eligible operators (34%). Final plan contains 75 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 332 out of 392 eligible operators (84%). Final plan contains 36 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q78.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q78.native_datafusion/extended.txt index af4b7000b9..ebfdcc3c62 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q78.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q78.native_datafusion/extended.txt @@ -14,15 +14,14 @@ TakeOrderedAndProject : : : +- CometFilter : : : +- CometSortMergeJoin : : : :- CometSort - : : : : +- CometColumnarExchange - : : : : +- Filter - : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometExchange + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometNativeColumnarToRow + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- CometSort : : : +- CometExchange : : : +- CometProject @@ -42,11 +41,14 @@ TakeOrderedAndProject : : +- CometFilter : : +- CometSortMergeJoin : : :- CometSort - : : : +- CometColumnarExchange - : : : +- Filter - : : : +- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : +- ReusedSubquery + : : : +- CometExchange + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.web_sales + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometNativeColumnarToRow + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- CometSort : : +- CometExchange : : +- CometProject @@ -66,11 +68,14 @@ TakeOrderedAndProject : +- CometFilter : +- CometSortMergeJoin : :- CometSort - : : +- CometColumnarExchange - : : +- Filter - : : +- ColumnarToRow - : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : +- ReusedSubquery + : : +- CometExchange + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : +- SubqueryBroadcast + : : +- BroadcastExchange + : : +- CometNativeColumnarToRow + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim : +- CometSort : +- CometExchange : +- CometProject @@ -80,4 +85,4 @@ TakeOrderedAndProject +- CometFilter +- CometNativeScan parquet spark_catalog.default.date_dim -Comet accelerated 64 out of 76 eligible operators (84%). Final plan contains 5 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 74 out of 82 eligible operators (90%). Final plan contains 4 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q80a.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q80a.native_datafusion/extended.txt index da58e863bd..ec566b42de 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q80a.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q80a.native_datafusion/extended.txt @@ -22,16 +22,15 @@ CometNativeColumnarToRow : : : : : :- CometProject : : : : : : +- CometSortMergeJoin : : : : : : :- CometSort - : : : : : : : +- CometColumnarExchange - : : : : : : : +- Filter - : : : : : : : +- ColumnarToRow - : : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : : : +- SubqueryBroadcast - : : : : : : : +- BroadcastExchange - : : : : : : : +- CometNativeColumnarToRow - : : : : : : : +- CometProject - : : : : : : : +- CometFilter - : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : : +- CometExchange + : : : : : : : +- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : : : : +- SubqueryBroadcast + : : : : : : : +- BroadcastExchange + : : : : : : : +- CometNativeColumnarToRow + : : : : : : : +- CometProject + : : : : : : : +- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : : : +- CometSort : : : : : : +- CometExchange : : : : : : +- CometProject @@ -67,11 +66,15 @@ CometNativeColumnarToRow : : : : : :- CometProject : : : : : : +- CometSortMergeJoin : : : : : : :- CometSort - : : : : : : : +- CometColumnarExchange - : : : : : : : +- Filter - : : : : : : : +- ColumnarToRow - : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : : : +- ReusedSubquery + : : : : : : : +- CometExchange + : : : : : : : +- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : : : : +- SubqueryBroadcast + : : : : : : : +- BroadcastExchange + : : : : : : : +- CometNativeColumnarToRow + : : : : : : : +- CometProject + : : : : : : : +- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : : : +- CometSort : : : : : : +- CometExchange : : : : : : +- CometProject @@ -107,11 +110,15 @@ CometNativeColumnarToRow : : : : :- CometProject : : : : : +- CometSortMergeJoin : : : : : :- CometSort - : : : : : : +- CometColumnarExchange - : : : : : : +- Filter - : : : : : : +- ColumnarToRow - : : : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : : +- ReusedSubquery + : : : : : : +- CometExchange + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.web_sales + : : : : : : +- SubqueryBroadcast + : : : : : : +- BroadcastExchange + : : : : : : +- CometNativeColumnarToRow + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : : +- CometSort : : : : : +- CometExchange : : : : : +- CometProject @@ -154,16 +161,15 @@ CometNativeColumnarToRow : : : : : :- CometProject : : : : : : +- CometSortMergeJoin : : : : : : :- CometSort - : : : : : : : +- CometColumnarExchange - : : : : : : : +- Filter - : : : : : : : +- ColumnarToRow - : : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : : : +- SubqueryBroadcast - : : : : : : : +- BroadcastExchange - : : : : : : : +- CometNativeColumnarToRow - : : : : : : : +- CometProject - : : : : : : : +- CometFilter - : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : : +- CometExchange + : : : : : : : +- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : : : : +- SubqueryBroadcast + : : : : : : : +- BroadcastExchange + : : : : : : : +- CometNativeColumnarToRow + : : : : : : : +- CometProject + : : : : : : : +- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : : : +- CometSort : : : : : : +- CometExchange : : : : : : +- CometProject @@ -199,11 +205,15 @@ CometNativeColumnarToRow : : : : : :- CometProject : : : : : : +- CometSortMergeJoin : : : : : : :- CometSort - : : : : : : : +- CometColumnarExchange - : : : : : : : +- Filter - : : : : : : : +- ColumnarToRow - : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : : : +- ReusedSubquery + : : : : : : : +- CometExchange + : : : : : : : +- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : : : : +- SubqueryBroadcast + : : : : : : : +- BroadcastExchange + : : : : : : : +- CometNativeColumnarToRow + : : : : : : : +- CometProject + : : : : : : : +- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : : : +- CometSort : : : : : : +- CometExchange : : : : : : +- CometProject @@ -239,11 +249,15 @@ CometNativeColumnarToRow : : : : :- CometProject : : : : : +- CometSortMergeJoin : : : : : :- CometSort - : : : : : : +- CometColumnarExchange - : : : : : : +- Filter - : : : : : : +- ColumnarToRow - : : : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : : +- ReusedSubquery + : : : : : : +- CometExchange + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.web_sales + : : : : : : +- SubqueryBroadcast + : : : : : : +- BroadcastExchange + : : : : : : +- CometNativeColumnarToRow + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : : +- CometSort : : : : : +- CometExchange : : : : : +- CometProject @@ -286,16 +300,15 @@ CometNativeColumnarToRow : : : : :- CometProject : : : : : +- CometSortMergeJoin : : : : : :- CometSort - : : : : : : +- CometColumnarExchange - : : : : : : +- Filter - : : : : : : +- ColumnarToRow - : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometNativeColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : +- CometExchange + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : : : +- SubqueryBroadcast + : : : : : : +- BroadcastExchange + : : : : : : +- CometNativeColumnarToRow + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : : +- CometSort : : : : : +- CometExchange : : : : : +- CometProject @@ -331,11 +344,15 @@ CometNativeColumnarToRow : : : : :- CometProject : : : : : +- CometSortMergeJoin : : : : : :- CometSort - : : : : : : +- CometColumnarExchange - : : : : : : +- Filter - : : : : : : +- ColumnarToRow - : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : : +- ReusedSubquery + : : : : : : +- CometExchange + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : : : +- SubqueryBroadcast + : : : : : : +- BroadcastExchange + : : : : : : +- CometNativeColumnarToRow + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : : +- CometSort : : : : : +- CometExchange : : : : : +- CometProject @@ -371,11 +388,15 @@ CometNativeColumnarToRow : : : :- CometProject : : : : +- CometSortMergeJoin : : : : :- CometSort - : : : : : +- CometColumnarExchange - : : : : : +- Filter - : : : : : +- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : +- ReusedSubquery + : : : : : +- CometExchange + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.web_sales + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometNativeColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : +- CometSort : : : : +- CometExchange : : : : +- CometProject @@ -398,4 +419,4 @@ CometNativeColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.promotion -Comet accelerated 356 out of 386 eligible operators (92%). Final plan contains 13 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 392 out of 410 eligible operators (95%). Final plan contains 10 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q86a.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q86a.native_datafusion/extended.txt index d56e25c3af..c1df720935 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q86a.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q86a.native_datafusion/extended.txt @@ -5,97 +5,83 @@ TakeOrderedAndProject +- CometSort +- CometExchange +- CometHashAggregate - +- CometColumnarExchange - +- HashAggregate - +- Union - :- HashAggregate - : +- CometNativeColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Filter - : : : +- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- CometNativeColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.item - :- HashAggregate - : +- CometNativeColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- HashAggregate - : +- CometNativeColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Filter - : : : +- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- CometNativeColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.item - +- HashAggregate - +- CometNativeColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- HashAggregate - +- CometNativeColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- Filter - : : +- ColumnarToRow - : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- CometNativeColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.date_dim - +- BroadcastExchange - +- CometNativeColumnarToRow - +- CometProject - +- CometFilter - +- CometNativeScan parquet spark_catalog.default.item + +- CometExchange + +- CometHashAggregate + +- CometUnion + :- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.web_sales + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometNativeColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.item + :- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.web_sales + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometNativeColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.item + +- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.web_sales + : : +- SubqueryBroadcast + : : +- BroadcastExchange + : : +- CometNativeColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.date_dim + +- CometBroadcastExchange + +- CometProject + +- CometFilter + +- CometNativeScan parquet spark_catalog.default.item -Comet accelerated 36 out of 81 eligible operators (44%). Final plan contains 18 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 72 out of 81 eligible operators (88%). Final plan contains 4 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q98.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q98.native_datafusion/extended.txt index b8d60879d2..6c2b043008 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q98.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q98.native_datafusion/extended.txt @@ -5,33 +5,29 @@ CometNativeColumnarToRow +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] +- CometNativeColumnarToRow +- CometSort - +- CometColumnarExchange - +- HashAggregate - +- CometNativeColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- Filter - : : +- ColumnarToRow - : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- CometNativeColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.item - +- BroadcastExchange - +- CometNativeColumnarToRow - +- CometProject - +- CometFilter - +- CometNativeScan parquet spark_catalog.default.date_dim + +- CometExchange + +- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : +- SubqueryBroadcast + : : +- BroadcastExchange + : : +- CometNativeColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.item + +- CometBroadcastExchange + +- CometProject + +- CometFilter + +- CometNativeScan parquet spark_catalog.default.date_dim -Comet accelerated 14 out of 28 eligible operators (50%). Final plan contains 7 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 24 out of 28 eligible operators (85%). Final plan contains 3 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q10a.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q10a.native_datafusion/extended.txt index 9091536e96..cba125f412 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q10a.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q10a.native_datafusion/extended.txt @@ -1,65 +1,66 @@ -TakeOrderedAndProject -+- HashAggregate - +- CometNativeColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- BroadcastHashJoin - : : : :- CometNativeColumnarToRow - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.customer - : : : +- BroadcastExchange - : : : +- Project - : : : +- BroadcastHashJoin - : : : :- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- Union - : : :- Project - : : : +- BroadcastHashJoin - : : : :- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- ReusedSubquery - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- Project - : : +- BroadcastHashJoin - : : :- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : +- ReusedSubquery - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- CometNativeColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.customer_address - +- BroadcastExchange - +- CometNativeColumnarToRow - +- CometProject - +- CometFilter - +- CometNativeScan parquet spark_catalog.default.customer_demographics +CometNativeColumnarToRow ++- CometTakeOrderedAndProject + +- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.customer + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometNativeScan parquet spark_catalog.default.store_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometNativeColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometUnion + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometNativeScan parquet spark_catalog.default.web_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometNativeColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometNativeColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.customer_address + +- CometBroadcastExchange + +- CometProject + +- CometFilter + +- CometNativeScan parquet spark_catalog.default.customer_demographics -Comet accelerated 21 out of 52 eligible operators (40%). Final plan contains 11 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 54 out of 60 eligible operators (90%). Final plan contains 4 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q11.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q11.native_datafusion/extended.txt index 5d218998be..51a3f5ca94 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q11.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q11.native_datafusion/extended.txt @@ -1,105 +1,98 @@ -TakeOrderedAndProject -+- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- BroadcastHashJoin - : : :- Filter - : : : +- HashAggregate - : : : +- CometNativeColumnarToRow - : : : +- CometColumnarExchange - : : : +- HashAggregate - : : : +- Project - : : : +- BroadcastHashJoin - : : : :- Project - : : : : +- BroadcastHashJoin - : : : : :- CometNativeColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.customer - : : : : +- BroadcastExchange - : : : : +- Filter - : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- HashAggregate - : : +- CometNativeColumnarToRow - : : +- CometColumnarExchange - : : +- HashAggregate - : : +- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.customer - : : : +- BroadcastExchange - : : : +- Filter - : : : +- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- Filter - : +- HashAggregate - : +- CometNativeColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.customer - : : +- BroadcastExchange - : : +- Filter - : : +- ColumnarToRow - : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : +- ReusedSubquery - : +- BroadcastExchange - : +- CometNativeColumnarToRow - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.date_dim - +- BroadcastExchange - +- HashAggregate - +- CometNativeColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- CometNativeColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.customer - : +- BroadcastExchange - : +- Filter - : +- ColumnarToRow - : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : +- ReusedSubquery - +- BroadcastExchange - +- CometNativeColumnarToRow - +- CometFilter - +- CometNativeScan parquet spark_catalog.default.date_dim +CometNativeColumnarToRow ++- CometTakeOrderedAndProject + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometHashAggregate + : : : +- CometExchange + : : : +- CometHashAggregate + : : : +- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.customer + : : : : +- CometBroadcastExchange + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometNativeColumnarToRow + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometHashAggregate + : : +- CometExchange + : : +- CometHashAggregate + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.customer + : : : +- CometBroadcastExchange + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometNativeColumnarToRow + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometFilter + : +- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.customer + : : +- CometBroadcastExchange + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.web_sales + : : +- SubqueryBroadcast + : : +- BroadcastExchange + : : +- CometNativeColumnarToRow + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.date_dim + +- CometBroadcastExchange + +- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.customer + : +- CometBroadcastExchange + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.web_sales + : +- SubqueryBroadcast + : +- BroadcastExchange + : +- CometNativeColumnarToRow + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.date_dim + +- CometBroadcastExchange + +- CometFilter + +- CometNativeScan parquet spark_catalog.default.date_dim -Comet accelerated 28 out of 85 eligible operators (32%). Final plan contains 18 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 83 out of 91 eligible operators (91%). Final plan contains 5 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q12.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q12.native_datafusion/extended.txt index 15648408fc..a8c2531a38 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q12.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q12.native_datafusion/extended.txt @@ -3,33 +3,29 @@ TakeOrderedAndProject +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] +- CometNativeColumnarToRow +- CometSort - +- CometColumnarExchange - +- HashAggregate - +- CometNativeColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- Filter - : : +- ColumnarToRow - : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- CometNativeColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.item - +- BroadcastExchange - +- CometNativeColumnarToRow - +- CometProject - +- CometFilter - +- CometNativeScan parquet spark_catalog.default.date_dim + +- CometExchange + +- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.web_sales + : : +- SubqueryBroadcast + : : +- BroadcastExchange + : : +- CometNativeColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.item + +- CometBroadcastExchange + +- CometProject + +- CometFilter + +- CometNativeScan parquet spark_catalog.default.date_dim -Comet accelerated 12 out of 27 eligible operators (44%). Final plan contains 6 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 22 out of 27 eligible operators (81%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q14.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q14.native_datafusion/extended.txt index 25e8321ad7..b48995c0a5 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q14.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q14.native_datafusion/extended.txt @@ -1,408 +1,414 @@ -TakeOrderedAndProject -+- BroadcastHashJoin - :- Filter - : : +- Subquery - : : +- HashAggregate - : : +- CometNativeColumnarToRow - : : +- CometColumnarExchange - : : +- HashAggregate - : : +- Union - : : :- Project - : : : +- BroadcastHashJoin - : : : :- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- ReusedSubquery - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : :- Project - : : : +- BroadcastHashJoin - : : : :- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- ReusedSubquery - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- Project - : : +- BroadcastHashJoin - : : :- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : +- ReusedSubquery - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- HashAggregate - : +- CometNativeColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- BroadcastHashJoin - : : : :- Filter - : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : : +- ReusedSubquery - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : +- Subquery - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- Project - : : : +- BroadcastHashJoin - : : : :- CometNativeColumnarToRow - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : +- BroadcastExchange - : : : +- BroadcastHashJoin - : : : :- CometNativeColumnarToRow - : : : : +- CometHashAggregate - : : : : +- CometColumnarExchange - : : : : +- HashAggregate - : : : : +- Project - : : : : +- BroadcastHashJoin - : : : : :- Project - : : : : : +- BroadcastHashJoin - : : : : : :- Filter - : : : : : : +- ColumnarToRow - : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometNativeColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : : +- BroadcastExchange - : : : : : +- BroadcastHashJoin - : : : : : :- CometNativeColumnarToRow - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : : +- BroadcastExchange - : : : : : +- Project - : : : : : +- BroadcastHashJoin - : : : : : :- Project - : : : : : : +- BroadcastHashJoin - : : : : : : :- Filter - : : : : : : : +- ColumnarToRow - : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : : : +- ReusedSubquery - : : : : : : +- BroadcastExchange - : : : : : : +- CometNativeColumnarToRow - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : : +- BroadcastExchange - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- Project - : : : +- BroadcastHashJoin - : : : :- Project - : : : : +- BroadcastHashJoin - : : : : :- Filter - : : : : : +- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : +- ReusedSubquery - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- BroadcastHashJoin - : : :- CometNativeColumnarToRow - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.item - : : +- BroadcastExchange - : : +- Project - : : +- BroadcastHashJoin - : : :- CometNativeColumnarToRow - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.item - : : +- BroadcastExchange - : : +- BroadcastHashJoin - : : :- CometNativeColumnarToRow - : : : +- CometHashAggregate - : : : +- CometColumnarExchange - : : : +- HashAggregate - : : : +- Project - : : : +- BroadcastHashJoin - : : : :- Project - : : : : +- BroadcastHashJoin - : : : : :- Filter - : : : : : +- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : +- BroadcastExchange - : : : : +- BroadcastHashJoin - : : : : :- CometNativeColumnarToRow - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : +- BroadcastExchange - : : : : +- Project - : : : : +- BroadcastHashJoin - : : : : :- Project - : : : : : +- BroadcastHashJoin - : : : : : :- Filter - : : : : : : +- ColumnarToRow - : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : : +- ReusedSubquery - : : : : : +- BroadcastExchange - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- Filter - : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- ReusedSubquery - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.item - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- CometNativeColumnarToRow - : +- CometProject - : +- CometFilter - : : +- ReusedSubquery - : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- Subquery - : +- CometNativeColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.date_dim - +- BroadcastExchange - +- Filter - : +- ReusedSubquery - +- HashAggregate - +- CometNativeColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- BroadcastHashJoin - : : :- Filter - : : : +- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : : +- ReusedSubquery - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- Subquery - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- Project - : : +- BroadcastHashJoin - : : :- CometNativeColumnarToRow - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.item - : : +- BroadcastExchange - : : +- BroadcastHashJoin - : : :- CometNativeColumnarToRow - : : : +- CometHashAggregate - : : : +- CometColumnarExchange - : : : +- HashAggregate - : : : +- Project - : : : +- BroadcastHashJoin - : : : :- Project - : : : : +- BroadcastHashJoin - : : : : :- Filter - : : : : : +- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : +- BroadcastExchange - : : : : +- BroadcastHashJoin - : : : : :- CometNativeColumnarToRow - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : +- BroadcastExchange - : : : : +- Project - : : : : +- BroadcastHashJoin - : : : : :- Project - : : : : : +- BroadcastHashJoin - : : : : : :- Filter - : : : : : : +- ColumnarToRow - : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : : +- ReusedSubquery - : : : : : +- BroadcastExchange - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- Filter - : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- ReusedSubquery - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.item - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- BroadcastHashJoin - : :- CometNativeColumnarToRow - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.item - : +- BroadcastExchange - : +- Project - : +- BroadcastHashJoin - : :- CometNativeColumnarToRow - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.item - : +- BroadcastExchange - : +- BroadcastHashJoin - : :- CometNativeColumnarToRow - : : +- CometHashAggregate - : : +- CometColumnarExchange - : : +- HashAggregate - : : +- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- Filter - : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- BroadcastHashJoin - : : : :- CometNativeColumnarToRow - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : +- BroadcastExchange - : : : +- Project - : : : +- BroadcastHashJoin - : : : :- Project - : : : : +- BroadcastHashJoin - : : : : :- Filter - : : : : : +- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : +- ReusedSubquery - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Filter - : : : +- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : +- ReusedSubquery - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.item - : +- BroadcastExchange - : +- CometNativeColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.date_dim - +- BroadcastExchange - +- CometNativeColumnarToRow - +- CometProject - +- CometFilter - : +- ReusedSubquery - +- CometNativeScan parquet spark_catalog.default.date_dim - +- Subquery - +- CometNativeColumnarToRow - +- CometProject - +- CometFilter - +- CometNativeScan parquet spark_catalog.default.date_dim +CometNativeColumnarToRow ++- CometTakeOrderedAndProject + +- CometBroadcastHashJoin + :- CometFilter + : : +- Subquery + : : +- CometNativeColumnarToRow + : : +- CometHashAggregate + : : +- CometExchange + : : +- CometHashAggregate + : : +- CometUnion + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometNativeScan parquet spark_catalog.default.store_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometNativeColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometNativeColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometNativeScan parquet spark_catalog.default.web_sales + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometNativeColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometNativeColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : : +- Subquery + : : : : : +- CometNativeColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- Subquery + : : : : +- CometNativeColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : +- CometBroadcastExchange + : : : +- CometBroadcastHashJoin + : : : :- CometHashAggregate + : : : : +- CometExchange + : : : : +- CometHashAggregate + : : : : +- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometProject + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : : : +- SubqueryBroadcast + : : : : : : +- BroadcastExchange + : : : : : : +- CometNativeColumnarToRow + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- CometBroadcastExchange + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : : : +- CometBroadcastExchange + : : : : : +- CometProject + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometProject + : : : : : : +- CometBroadcastHashJoin + : : : : : : :- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : : : : +- SubqueryBroadcast + : : : : : : : +- BroadcastExchange + : : : : : : : +- CometNativeColumnarToRow + : : : : : : : +- CometProject + : : : : : : : +- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : +- CometBroadcastExchange + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : : : +- CometBroadcastExchange + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.web_sales + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometNativeColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.item + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.item + : : +- CometBroadcastExchange + : : +- CometBroadcastHashJoin + : : :- CometHashAggregate + : : : +- CometExchange + : : : +- CometHashAggregate + : : : +- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometNativeColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometBroadcastHashJoin + : : : : :- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometProject + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : : : +- SubqueryBroadcast + : : : : : : +- BroadcastExchange + : : : : : : +- CometNativeColumnarToRow + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- CometBroadcastExchange + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.web_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometNativeColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.item + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : : +- ReusedSubquery + : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- Subquery + : +- CometNativeColumnarToRow + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.date_dim + +- CometBroadcastExchange + +- CometFilter + : +- ReusedSubquery + +- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometNativeColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : : +- Subquery + : : : : +- CometNativeColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- Subquery + : : : +- CometNativeColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.item + : : +- CometBroadcastExchange + : : +- CometBroadcastHashJoin + : : :- CometHashAggregate + : : : +- CometExchange + : : : +- CometHashAggregate + : : : +- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometNativeColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometBroadcastHashJoin + : : : : :- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometProject + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : : : +- SubqueryBroadcast + : : : : : : +- BroadcastExchange + : : : : : : +- CometNativeColumnarToRow + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- CometBroadcastExchange + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.web_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometNativeColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.item + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometBroadcastHashJoin + : :- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.item + : +- CometBroadcastExchange + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.item + : +- CometBroadcastExchange + : +- CometBroadcastHashJoin + : :- CometHashAggregate + : : +- CometExchange + : : +- CometHashAggregate + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometNativeColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometNativeColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.web_sales + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometNativeColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.item + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.date_dim + +- CometBroadcastExchange + +- CometProject + +- CometFilter + : +- ReusedSubquery + +- CometNativeScan parquet spark_catalog.default.date_dim + +- Subquery + +- CometNativeColumnarToRow + +- CometProject + +- CometFilter + +- CometNativeScan parquet spark_catalog.default.date_dim -Comet accelerated 128 out of 337 eligible operators (37%). Final plan contains 69 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 343 out of 387 eligible operators (88%). Final plan contains 25 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q14a.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q14a.native_datafusion/extended.txt index 0bc07e401e..636e5ce377 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q14a.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q14a.native_datafusion/extended.txt @@ -1,2779 +1,2750 @@ CometNativeColumnarToRow +- CometTakeOrderedAndProject +- CometHashAggregate - +- CometColumnarExchange - +- HashAggregate - +- Union - :- HashAggregate - : +- CometNativeColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Union - : :- Filter - : : : +- Subquery - : : : +- HashAggregate - : : : +- CometNativeColumnarToRow - : : : +- CometColumnarExchange - : : : +- HashAggregate - : : : +- Union - : : : :- Project - : : : : +- BroadcastHashJoin - : : : : :- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : +- ReusedSubquery - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : :- Project - : : : : +- BroadcastHashJoin - : : : : :- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- Project - : : : +- BroadcastHashJoin - : : : :- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- ReusedSubquery - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- HashAggregate - : : +- CometNativeColumnarToRow - : : +- CometColumnarExchange - : : +- HashAggregate - : : +- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- BroadcastHashJoin - : : : : :- Filter - : : : : : +- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : +- BroadcastExchange - : : : : +- Project - : : : : +- BroadcastHashJoin - : : : : :- CometNativeColumnarToRow - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : +- BroadcastExchange - : : : : +- BroadcastHashJoin - : : : : :- CometNativeColumnarToRow - : : : : : +- CometHashAggregate - : : : : : +- CometColumnarExchange - : : : : : +- HashAggregate - : : : : : +- Project - : : : : : +- BroadcastHashJoin - : : : : : :- Project - : : : : : : +- BroadcastHashJoin - : : : : : : :- Filter - : : : : : : : +- ColumnarToRow - : : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : : : +- SubqueryBroadcast - : : : : : : : +- BroadcastExchange - : : : : : : : +- CometNativeColumnarToRow - : : : : : : : +- CometProject - : : : : : : : +- CometFilter - : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : : : +- BroadcastExchange - : : : : : : +- BroadcastHashJoin - : : : : : : :- CometNativeColumnarToRow - : : : : : : : +- CometFilter - : : : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : : : +- BroadcastExchange - : : : : : : +- Project - : : : : : : +- BroadcastHashJoin - : : : : : : :- Project - : : : : : : : +- BroadcastHashJoin - : : : : : : : :- Filter - : : : : : : : : +- ColumnarToRow - : : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : : : : +- ReusedSubquery - : : : : : : : +- BroadcastExchange - : : : : : : : +- CometNativeColumnarToRow - : : : : : : : +- CometFilter - : : : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : : : +- BroadcastExchange - : : : : : : +- CometNativeColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : : +- BroadcastExchange - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : +- BroadcastExchange - : : : : +- Project - : : : : +- BroadcastHashJoin - : : : : :- Project - : : : : : +- BroadcastHashJoin - : : : : : :- Filter - : : : : : : +- ColumnarToRow - : : : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : : +- ReusedSubquery - : : : : : +- BroadcastExchange - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- BroadcastHashJoin - : : : :- CometNativeColumnarToRow - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : +- BroadcastExchange - : : : +- Project - : : : +- BroadcastHashJoin - : : : :- CometNativeColumnarToRow - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : +- BroadcastExchange - : : : +- BroadcastHashJoin - : : : :- CometNativeColumnarToRow - : : : : +- CometHashAggregate - : : : : +- CometColumnarExchange - : : : : +- HashAggregate - : : : : +- Project - : : : : +- BroadcastHashJoin - : : : : :- Project - : : : : : +- BroadcastHashJoin - : : : : : :- Filter - : : : : : : +- ColumnarToRow - : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometNativeColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : : +- BroadcastExchange - : : : : : +- BroadcastHashJoin - : : : : : :- CometNativeColumnarToRow - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : : +- BroadcastExchange - : : : : : +- Project - : : : : : +- BroadcastHashJoin - : : : : : :- Project - : : : : : : +- BroadcastHashJoin - : : : : : : :- Filter - : : : : : : : +- ColumnarToRow - : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : : : +- ReusedSubquery - : : : : : : +- BroadcastExchange - : : : : : : +- CometNativeColumnarToRow - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : : +- BroadcastExchange - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- Project - : : : +- BroadcastHashJoin - : : : :- Project - : : : : +- BroadcastHashJoin - : : : : :- Filter - : : : : : +- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : +- ReusedSubquery - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : :- Filter - : : : +- ReusedSubquery - : : +- HashAggregate - : : +- CometNativeColumnarToRow - : : +- CometColumnarExchange - : : +- HashAggregate - : : +- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- BroadcastHashJoin - : : : : :- Filter - : : : : : +- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : +- ReusedSubquery - : : : : +- BroadcastExchange - : : : : +- Project - : : : : +- BroadcastHashJoin - : : : : :- CometNativeColumnarToRow - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : +- BroadcastExchange - : : : : +- BroadcastHashJoin - : : : : :- CometNativeColumnarToRow - : : : : : +- CometHashAggregate - : : : : : +- CometColumnarExchange - : : : : : +- HashAggregate - : : : : : +- Project - : : : : : +- BroadcastHashJoin - : : : : : :- Project - : : : : : : +- BroadcastHashJoin - : : : : : : :- Filter - : : : : : : : +- ColumnarToRow - : : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : : : +- SubqueryBroadcast - : : : : : : : +- BroadcastExchange - : : : : : : : +- CometNativeColumnarToRow - : : : : : : : +- CometProject - : : : : : : : +- CometFilter - : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : : : +- BroadcastExchange - : : : : : : +- BroadcastHashJoin - : : : : : : :- CometNativeColumnarToRow - : : : : : : : +- CometFilter - : : : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : : : +- BroadcastExchange - : : : : : : +- Project - : : : : : : +- BroadcastHashJoin - : : : : : : :- Project - : : : : : : : +- BroadcastHashJoin - : : : : : : : :- Filter - : : : : : : : : +- ColumnarToRow - : : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : : : : +- ReusedSubquery - : : : : : : : +- BroadcastExchange - : : : : : : : +- CometNativeColumnarToRow - : : : : : : : +- CometFilter - : : : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : : : +- BroadcastExchange - : : : : : : +- CometNativeColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : : +- BroadcastExchange - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : +- BroadcastExchange - : : : : +- Project - : : : : +- BroadcastHashJoin - : : : : :- Project - : : : : : +- BroadcastHashJoin - : : : : : :- Filter - : : : : : : +- ColumnarToRow - : : : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : : +- ReusedSubquery - : : : : : +- BroadcastExchange - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- BroadcastHashJoin - : : : :- CometNativeColumnarToRow - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : +- BroadcastExchange - : : : +- Project - : : : +- BroadcastHashJoin - : : : :- CometNativeColumnarToRow - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : +- BroadcastExchange - : : : +- BroadcastHashJoin - : : : :- CometNativeColumnarToRow - : : : : +- CometHashAggregate - : : : : +- CometColumnarExchange - : : : : +- HashAggregate - : : : : +- Project - : : : : +- BroadcastHashJoin - : : : : :- Project - : : : : : +- BroadcastHashJoin - : : : : : :- Filter - : : : : : : +- ColumnarToRow - : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometNativeColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : : +- BroadcastExchange - : : : : : +- BroadcastHashJoin - : : : : : :- CometNativeColumnarToRow - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : : +- BroadcastExchange - : : : : : +- Project - : : : : : +- BroadcastHashJoin - : : : : : :- Project - : : : : : : +- BroadcastHashJoin - : : : : : : :- Filter - : : : : : : : +- ColumnarToRow - : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : : : +- ReusedSubquery - : : : : : : +- BroadcastExchange - : : : : : : +- CometNativeColumnarToRow - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : : +- BroadcastExchange - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- Project - : : : +- BroadcastHashJoin - : : : :- Project - : : : : +- BroadcastHashJoin - : : : : :- Filter - : : : : : +- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : +- ReusedSubquery - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- Filter - : : +- ReusedSubquery - : +- HashAggregate - : +- CometNativeColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- BroadcastHashJoin - : : : :- Filter - : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- ReusedSubquery - : : : +- BroadcastExchange - : : : +- Project - : : : +- BroadcastHashJoin - : : : :- CometNativeColumnarToRow - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : +- BroadcastExchange - : : : +- BroadcastHashJoin - : : : :- CometNativeColumnarToRow - : : : : +- CometHashAggregate - : : : : +- CometColumnarExchange - : : : : +- HashAggregate - : : : : +- Project - : : : : +- BroadcastHashJoin - : : : : :- Project - : : : : : +- BroadcastHashJoin - : : : : : :- Filter - : : : : : : +- ColumnarToRow - : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometNativeColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : : +- BroadcastExchange - : : : : : +- BroadcastHashJoin - : : : : : :- CometNativeColumnarToRow - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : : +- BroadcastExchange - : : : : : +- Project - : : : : : +- BroadcastHashJoin - : : : : : :- Project - : : : : : : +- BroadcastHashJoin - : : : : : : :- Filter - : : : : : : : +- ColumnarToRow - : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : : : +- ReusedSubquery - : : : : : : +- BroadcastExchange - : : : : : : +- CometNativeColumnarToRow - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : : +- BroadcastExchange - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- Project - : : : +- BroadcastHashJoin - : : : :- Project - : : : : +- BroadcastHashJoin - : : : : :- Filter - : : : : : +- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : +- ReusedSubquery - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- BroadcastHashJoin - : : :- CometNativeColumnarToRow - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.item - : : +- BroadcastExchange - : : +- Project - : : +- BroadcastHashJoin - : : :- CometNativeColumnarToRow - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.item - : : +- BroadcastExchange - : : +- BroadcastHashJoin - : : :- CometNativeColumnarToRow - : : : +- CometHashAggregate - : : : +- CometColumnarExchange - : : : +- HashAggregate - : : : +- Project - : : : +- BroadcastHashJoin - : : : :- Project - : : : : +- BroadcastHashJoin - : : : : :- Filter - : : : : : +- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : +- BroadcastExchange - : : : : +- BroadcastHashJoin - : : : : :- CometNativeColumnarToRow - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : +- BroadcastExchange - : : : : +- Project - : : : : +- BroadcastHashJoin - : : : : :- Project - : : : : : +- BroadcastHashJoin - : : : : : :- Filter - : : : : : : +- ColumnarToRow - : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : : +- ReusedSubquery - : : : : : +- BroadcastExchange - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- Filter - : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- ReusedSubquery - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.item - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- CometNativeColumnarToRow + +- CometExchange + +- CometHashAggregate + +- CometUnion + :- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometUnion + : :- CometFilter + : : : +- Subquery + : : : +- CometNativeColumnarToRow + : : : +- CometHashAggregate + : : : +- CometExchange + : : : +- CometHashAggregate + : : : +- CometUnion + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometNativeScan parquet spark_catalog.default.store_sales + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometNativeColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometNativeColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometNativeScan parquet spark_catalog.default.web_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometNativeColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometHashAggregate + : : +- CometExchange + : : +- CometHashAggregate + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometBroadcastHashJoin + : : : : :- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometNativeColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : : +- CometBroadcastExchange + : : : : +- CometBroadcastHashJoin + : : : : :- CometHashAggregate + : : : : : +- CometExchange + : : : : : +- CometHashAggregate + : : : : : +- CometProject + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometProject + : : : : : : +- CometBroadcastHashJoin + : : : : : : :- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : : : : +- SubqueryBroadcast + : : : : : : : +- BroadcastExchange + : : : : : : : +- CometNativeColumnarToRow + : : : : : : : +- CometProject + : : : : : : : +- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : +- CometBroadcastExchange + : : : : : : +- CometBroadcastHashJoin + : : : : : : :- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : : : : +- CometBroadcastExchange + : : : : : : +- CometProject + : : : : : : +- CometBroadcastHashJoin + : : : : : : :- CometProject + : : : : : : : +- CometBroadcastHashJoin + : : : : : : : :- CometFilter + : : : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : : : : : +- SubqueryBroadcast + : : : : : : : : +- BroadcastExchange + : : : : : : : : +- CometNativeColumnarToRow + : : : : : : : : +- CometProject + : : : : : : : : +- CometFilter + : : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : : +- CometBroadcastExchange + : : : : : : : +- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : : : : +- CometBroadcastExchange + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- CometBroadcastExchange + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometProject + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.web_sales + : : : : : : +- SubqueryBroadcast + : : : : : : +- BroadcastExchange + : : : : : : +- CometNativeColumnarToRow + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- CometBroadcastExchange + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : +- CometBroadcastExchange + : : : +- CometBroadcastHashJoin + : : : :- CometHashAggregate + : : : : +- CometExchange + : : : : +- CometHashAggregate + : : : : +- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometProject + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : : : +- SubqueryBroadcast + : : : : : : +- BroadcastExchange + : : : : : : +- CometNativeColumnarToRow + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- CometBroadcastExchange + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : : : +- CometBroadcastExchange + : : : : : +- CometProject + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometProject + : : : : : : +- CometBroadcastHashJoin + : : : : : : :- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : : : : +- SubqueryBroadcast + : : : : : : : +- BroadcastExchange + : : : : : : : +- CometNativeColumnarToRow + : : : : : : : +- CometProject + : : : : : : : +- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : +- CometBroadcastExchange + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : : : +- CometBroadcastExchange + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.web_sales + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometNativeColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : :- CometFilter + : : : +- ReusedSubquery + : : +- CometHashAggregate + : : +- CometExchange + : : +- CometHashAggregate + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometBroadcastHashJoin + : : : : :- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometNativeColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : : +- CometBroadcastExchange + : : : : +- CometBroadcastHashJoin + : : : : :- CometHashAggregate + : : : : : +- CometExchange + : : : : : +- CometHashAggregate + : : : : : +- CometProject + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometProject + : : : : : : +- CometBroadcastHashJoin + : : : : : : :- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : : : : +- SubqueryBroadcast + : : : : : : : +- BroadcastExchange + : : : : : : : +- CometNativeColumnarToRow + : : : : : : : +- CometProject + : : : : : : : +- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : +- CometBroadcastExchange + : : : : : : +- CometBroadcastHashJoin + : : : : : : :- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : : : : +- CometBroadcastExchange + : : : : : : +- CometProject + : : : : : : +- CometBroadcastHashJoin + : : : : : : :- CometProject + : : : : : : : +- CometBroadcastHashJoin + : : : : : : : :- CometFilter + : : : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : : : : : +- SubqueryBroadcast + : : : : : : : : +- BroadcastExchange + : : : : : : : : +- CometNativeColumnarToRow + : : : : : : : : +- CometProject + : : : : : : : : +- CometFilter + : : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : : +- CometBroadcastExchange + : : : : : : : +- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : : : : +- CometBroadcastExchange + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- CometBroadcastExchange + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometProject + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.web_sales + : : : : : : +- SubqueryBroadcast + : : : : : : +- BroadcastExchange + : : : : : : +- CometNativeColumnarToRow + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- CometBroadcastExchange + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : +- CometBroadcastExchange + : : : +- CometBroadcastHashJoin + : : : :- CometHashAggregate + : : : : +- CometExchange + : : : : +- CometHashAggregate + : : : : +- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometProject + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : : : +- SubqueryBroadcast + : : : : : : +- BroadcastExchange + : : : : : : +- CometNativeColumnarToRow + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- CometBroadcastExchange + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : : : +- CometBroadcastExchange + : : : : : +- CometProject + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometProject + : : : : : : +- CometBroadcastHashJoin + : : : : : : :- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : : : : +- SubqueryBroadcast + : : : : : : : +- BroadcastExchange + : : : : : : : +- CometNativeColumnarToRow + : : : : : : : +- CometProject + : : : : : : : +- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : +- CometBroadcastExchange + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : : : +- CometBroadcastExchange + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.web_sales + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometNativeColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometFilter + : : +- ReusedSubquery + : +- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.web_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometNativeColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : +- CometBroadcastExchange + : : : +- CometBroadcastHashJoin + : : : :- CometHashAggregate + : : : : +- CometExchange + : : : : +- CometHashAggregate + : : : : +- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometProject + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : : : +- SubqueryBroadcast + : : : : : : +- BroadcastExchange + : : : : : : +- CometNativeColumnarToRow + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- CometBroadcastExchange + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : : : +- CometBroadcastExchange + : : : : : +- CometProject + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometProject + : : : : : : +- CometBroadcastHashJoin + : : : : : : :- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : : : : +- SubqueryBroadcast + : : : : : : : +- BroadcastExchange + : : : : : : : +- CometNativeColumnarToRow + : : : : : : : +- CometProject + : : : : : : : +- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : +- CometBroadcastExchange + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : : : +- CometBroadcastExchange + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.web_sales + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometNativeColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.item + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.item + : : +- CometBroadcastExchange + : : +- CometBroadcastHashJoin + : : :- CometHashAggregate + : : : +- CometExchange + : : : +- CometHashAggregate + : : : +- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometNativeColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometBroadcastHashJoin + : : : : :- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometProject + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : : : +- SubqueryBroadcast + : : : : : : +- BroadcastExchange + : : : : : : +- CometNativeColumnarToRow + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- CometBroadcastExchange + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.web_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometNativeColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.item + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.date_dim + :- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometUnion + : :- CometFilter + : : : +- Subquery + : : : +- CometNativeColumnarToRow + : : : +- CometHashAggregate + : : : +- CometExchange + : : : +- CometHashAggregate + : : : +- CometUnion + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometNativeScan parquet spark_catalog.default.store_sales + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometNativeColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometNativeColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometNativeScan parquet spark_catalog.default.web_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometNativeColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometHashAggregate + : : +- CometExchange + : : +- CometHashAggregate + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometBroadcastHashJoin + : : : : :- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometNativeColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : : +- CometBroadcastExchange + : : : : +- CometBroadcastHashJoin + : : : : :- CometHashAggregate + : : : : : +- CometExchange + : : : : : +- CometHashAggregate + : : : : : +- CometProject + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometProject + : : : : : : +- CometBroadcastHashJoin + : : : : : : :- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : : : : +- SubqueryBroadcast + : : : : : : : +- BroadcastExchange + : : : : : : : +- CometNativeColumnarToRow + : : : : : : : +- CometProject + : : : : : : : +- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : +- CometBroadcastExchange + : : : : : : +- CometBroadcastHashJoin + : : : : : : :- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : : : : +- CometBroadcastExchange + : : : : : : +- CometProject + : : : : : : +- CometBroadcastHashJoin + : : : : : : :- CometProject + : : : : : : : +- CometBroadcastHashJoin + : : : : : : : :- CometFilter + : : : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : : : : : +- SubqueryBroadcast + : : : : : : : : +- BroadcastExchange + : : : : : : : : +- CometNativeColumnarToRow + : : : : : : : : +- CometProject + : : : : : : : : +- CometFilter + : : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : : +- CometBroadcastExchange + : : : : : : : +- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : : : : +- CometBroadcastExchange + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- CometBroadcastExchange + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometProject + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.web_sales + : : : : : : +- SubqueryBroadcast + : : : : : : +- BroadcastExchange + : : : : : : +- CometNativeColumnarToRow + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- CometBroadcastExchange + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : +- CometBroadcastExchange + : : : +- CometBroadcastHashJoin + : : : :- CometHashAggregate + : : : : +- CometExchange + : : : : +- CometHashAggregate + : : : : +- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometProject + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : : : +- SubqueryBroadcast + : : : : : : +- BroadcastExchange + : : : : : : +- CometNativeColumnarToRow + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- CometBroadcastExchange + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : : : +- CometBroadcastExchange + : : : : : +- CometProject + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometProject + : : : : : : +- CometBroadcastHashJoin + : : : : : : :- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : : : : +- SubqueryBroadcast + : : : : : : : +- BroadcastExchange + : : : : : : : +- CometNativeColumnarToRow + : : : : : : : +- CometProject + : : : : : : : +- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : +- CometBroadcastExchange + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : : : +- CometBroadcastExchange + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.web_sales + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometNativeColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : :- CometFilter + : : : +- ReusedSubquery + : : +- CometHashAggregate + : : +- CometExchange + : : +- CometHashAggregate + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometBroadcastHashJoin + : : : : :- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometNativeColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : : +- CometBroadcastExchange + : : : : +- CometBroadcastHashJoin + : : : : :- CometHashAggregate + : : : : : +- CometExchange + : : : : : +- CometHashAggregate + : : : : : +- CometProject + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometProject + : : : : : : +- CometBroadcastHashJoin + : : : : : : :- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : : : : +- SubqueryBroadcast + : : : : : : : +- BroadcastExchange + : : : : : : : +- CometNativeColumnarToRow + : : : : : : : +- CometProject + : : : : : : : +- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : +- CometBroadcastExchange + : : : : : : +- CometBroadcastHashJoin + : : : : : : :- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : : : : +- CometBroadcastExchange + : : : : : : +- CometProject + : : : : : : +- CometBroadcastHashJoin + : : : : : : :- CometProject + : : : : : : : +- CometBroadcastHashJoin + : : : : : : : :- CometFilter + : : : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : : : : : +- SubqueryBroadcast + : : : : : : : : +- BroadcastExchange + : : : : : : : : +- CometNativeColumnarToRow + : : : : : : : : +- CometProject + : : : : : : : : +- CometFilter + : : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : : +- CometBroadcastExchange + : : : : : : : +- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : : : : +- CometBroadcastExchange + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- CometBroadcastExchange + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometProject + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.web_sales + : : : : : : +- SubqueryBroadcast + : : : : : : +- BroadcastExchange + : : : : : : +- CometNativeColumnarToRow + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- CometBroadcastExchange + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : +- CometBroadcastExchange + : : : +- CometBroadcastHashJoin + : : : :- CometHashAggregate + : : : : +- CometExchange + : : : : +- CometHashAggregate + : : : : +- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometProject + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : : : +- SubqueryBroadcast + : : : : : : +- BroadcastExchange + : : : : : : +- CometNativeColumnarToRow + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- CometBroadcastExchange + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : : : +- CometBroadcastExchange + : : : : : +- CometProject + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometProject + : : : : : : +- CometBroadcastHashJoin + : : : : : : :- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : : : : +- SubqueryBroadcast + : : : : : : : +- BroadcastExchange + : : : : : : : +- CometNativeColumnarToRow + : : : : : : : +- CometProject + : : : : : : : +- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : +- CometBroadcastExchange + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : : : +- CometBroadcastExchange + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.web_sales + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometNativeColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometFilter + : : +- ReusedSubquery + : +- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.web_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometNativeColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : +- CometBroadcastExchange + : : : +- CometBroadcastHashJoin + : : : :- CometHashAggregate + : : : : +- CometExchange + : : : : +- CometHashAggregate + : : : : +- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometProject + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : : : +- SubqueryBroadcast + : : : : : : +- BroadcastExchange + : : : : : : +- CometNativeColumnarToRow + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- CometBroadcastExchange + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : : : +- CometBroadcastExchange + : : : : : +- CometProject + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometProject + : : : : : : +- CometBroadcastHashJoin + : : : : : : :- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : : : : +- SubqueryBroadcast + : : : : : : : +- BroadcastExchange + : : : : : : : +- CometNativeColumnarToRow + : : : : : : : +- CometProject + : : : : : : : +- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : +- CometBroadcastExchange + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : : : +- CometBroadcastExchange + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.web_sales + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometNativeColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.item + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.item + : : +- CometBroadcastExchange + : : +- CometBroadcastHashJoin + : : :- CometHashAggregate + : : : +- CometExchange + : : : +- CometHashAggregate + : : : +- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometNativeColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometBroadcastHashJoin + : : : : :- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometProject + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : : : +- SubqueryBroadcast + : : : : : : +- BroadcastExchange + : : : : : : +- CometNativeColumnarToRow + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- CometBroadcastExchange + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.web_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometNativeColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.item + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange : +- CometProject : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.date_dim - :- HashAggregate - : +- CometNativeColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- HashAggregate - : +- CometNativeColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Union - : :- Filter - : : : +- Subquery - : : : +- HashAggregate - : : : +- CometNativeColumnarToRow - : : : +- CometColumnarExchange - : : : +- HashAggregate - : : : +- Union - : : : :- Project - : : : : +- BroadcastHashJoin - : : : : :- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : +- ReusedSubquery - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : :- Project - : : : : +- BroadcastHashJoin - : : : : :- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- Project - : : : +- BroadcastHashJoin - : : : :- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- ReusedSubquery - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- HashAggregate - : : +- CometNativeColumnarToRow - : : +- CometColumnarExchange - : : +- HashAggregate - : : +- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- BroadcastHashJoin - : : : : :- Filter - : : : : : +- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : +- BroadcastExchange - : : : : +- Project - : : : : +- BroadcastHashJoin - : : : : :- CometNativeColumnarToRow - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : +- BroadcastExchange - : : : : +- BroadcastHashJoin - : : : : :- CometNativeColumnarToRow - : : : : : +- CometHashAggregate - : : : : : +- CometColumnarExchange - : : : : : +- HashAggregate - : : : : : +- Project - : : : : : +- BroadcastHashJoin - : : : : : :- Project - : : : : : : +- BroadcastHashJoin - : : : : : : :- Filter - : : : : : : : +- ColumnarToRow - : : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : : : +- SubqueryBroadcast - : : : : : : : +- BroadcastExchange - : : : : : : : +- CometNativeColumnarToRow - : : : : : : : +- CometProject - : : : : : : : +- CometFilter - : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : : : +- BroadcastExchange - : : : : : : +- BroadcastHashJoin - : : : : : : :- CometNativeColumnarToRow - : : : : : : : +- CometFilter - : : : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : : : +- BroadcastExchange - : : : : : : +- Project - : : : : : : +- BroadcastHashJoin - : : : : : : :- Project - : : : : : : : +- BroadcastHashJoin - : : : : : : : :- Filter - : : : : : : : : +- ColumnarToRow - : : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : : : : +- ReusedSubquery - : : : : : : : +- BroadcastExchange - : : : : : : : +- CometNativeColumnarToRow - : : : : : : : +- CometFilter - : : : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : : : +- BroadcastExchange - : : : : : : +- CometNativeColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : : +- BroadcastExchange - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : +- BroadcastExchange - : : : : +- Project - : : : : +- BroadcastHashJoin - : : : : :- Project - : : : : : +- BroadcastHashJoin - : : : : : :- Filter - : : : : : : +- ColumnarToRow - : : : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : : +- ReusedSubquery - : : : : : +- BroadcastExchange - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- BroadcastHashJoin - : : : :- CometNativeColumnarToRow - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : +- BroadcastExchange - : : : +- Project - : : : +- BroadcastHashJoin - : : : :- CometNativeColumnarToRow - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : +- BroadcastExchange - : : : +- BroadcastHashJoin - : : : :- CometNativeColumnarToRow - : : : : +- CometHashAggregate - : : : : +- CometColumnarExchange - : : : : +- HashAggregate - : : : : +- Project - : : : : +- BroadcastHashJoin - : : : : :- Project - : : : : : +- BroadcastHashJoin - : : : : : :- Filter - : : : : : : +- ColumnarToRow - : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometNativeColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : : +- BroadcastExchange - : : : : : +- BroadcastHashJoin - : : : : : :- CometNativeColumnarToRow - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : : +- BroadcastExchange - : : : : : +- Project - : : : : : +- BroadcastHashJoin - : : : : : :- Project - : : : : : : +- BroadcastHashJoin - : : : : : : :- Filter - : : : : : : : +- ColumnarToRow - : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : : : +- ReusedSubquery - : : : : : : +- BroadcastExchange - : : : : : : +- CometNativeColumnarToRow - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : : +- BroadcastExchange - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- Project - : : : +- BroadcastHashJoin - : : : :- Project - : : : : +- BroadcastHashJoin - : : : : :- Filter - : : : : : +- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : +- ReusedSubquery - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : :- Filter - : : : +- ReusedSubquery - : : +- HashAggregate - : : +- CometNativeColumnarToRow - : : +- CometColumnarExchange - : : +- HashAggregate - : : +- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- BroadcastHashJoin - : : : : :- Filter - : : : : : +- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : +- ReusedSubquery - : : : : +- BroadcastExchange - : : : : +- Project - : : : : +- BroadcastHashJoin - : : : : :- CometNativeColumnarToRow - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : +- BroadcastExchange - : : : : +- BroadcastHashJoin - : : : : :- CometNativeColumnarToRow - : : : : : +- CometHashAggregate - : : : : : +- CometColumnarExchange - : : : : : +- HashAggregate - : : : : : +- Project - : : : : : +- BroadcastHashJoin - : : : : : :- Project - : : : : : : +- BroadcastHashJoin - : : : : : : :- Filter - : : : : : : : +- ColumnarToRow - : : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : : : +- SubqueryBroadcast - : : : : : : : +- BroadcastExchange - : : : : : : : +- CometNativeColumnarToRow - : : : : : : : +- CometProject - : : : : : : : +- CometFilter - : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : : : +- BroadcastExchange - : : : : : : +- BroadcastHashJoin - : : : : : : :- CometNativeColumnarToRow - : : : : : : : +- CometFilter - : : : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : : : +- BroadcastExchange - : : : : : : +- Project - : : : : : : +- BroadcastHashJoin - : : : : : : :- Project - : : : : : : : +- BroadcastHashJoin - : : : : : : : :- Filter - : : : : : : : : +- ColumnarToRow - : : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : : : : +- ReusedSubquery - : : : : : : : +- BroadcastExchange - : : : : : : : +- CometNativeColumnarToRow - : : : : : : : +- CometFilter - : : : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : : : +- BroadcastExchange - : : : : : : +- CometNativeColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : : +- BroadcastExchange - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : +- BroadcastExchange - : : : : +- Project - : : : : +- BroadcastHashJoin - : : : : :- Project - : : : : : +- BroadcastHashJoin - : : : : : :- Filter - : : : : : : +- ColumnarToRow - : : : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : : +- ReusedSubquery - : : : : : +- BroadcastExchange - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- BroadcastHashJoin - : : : :- CometNativeColumnarToRow - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : +- BroadcastExchange - : : : +- Project - : : : +- BroadcastHashJoin - : : : :- CometNativeColumnarToRow - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : +- BroadcastExchange - : : : +- BroadcastHashJoin - : : : :- CometNativeColumnarToRow - : : : : +- CometHashAggregate - : : : : +- CometColumnarExchange - : : : : +- HashAggregate - : : : : +- Project - : : : : +- BroadcastHashJoin - : : : : :- Project - : : : : : +- BroadcastHashJoin - : : : : : :- Filter - : : : : : : +- ColumnarToRow - : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometNativeColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : : +- BroadcastExchange - : : : : : +- BroadcastHashJoin - : : : : : :- CometNativeColumnarToRow - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : : +- BroadcastExchange - : : : : : +- Project - : : : : : +- BroadcastHashJoin - : : : : : :- Project - : : : : : : +- BroadcastHashJoin - : : : : : : :- Filter - : : : : : : : +- ColumnarToRow - : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : : : +- ReusedSubquery - : : : : : : +- BroadcastExchange - : : : : : : +- CometNativeColumnarToRow - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : : +- BroadcastExchange - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- Project - : : : +- BroadcastHashJoin - : : : :- Project - : : : : +- BroadcastHashJoin - : : : : :- Filter - : : : : : +- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : +- ReusedSubquery - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- Filter - : : +- ReusedSubquery - : +- HashAggregate - : +- CometNativeColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- BroadcastHashJoin - : : : :- Filter - : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- ReusedSubquery - : : : +- BroadcastExchange - : : : +- Project - : : : +- BroadcastHashJoin - : : : :- CometNativeColumnarToRow - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : +- BroadcastExchange - : : : +- BroadcastHashJoin - : : : :- CometNativeColumnarToRow - : : : : +- CometHashAggregate - : : : : +- CometColumnarExchange - : : : : +- HashAggregate - : : : : +- Project - : : : : +- BroadcastHashJoin - : : : : :- Project - : : : : : +- BroadcastHashJoin - : : : : : :- Filter - : : : : : : +- ColumnarToRow - : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometNativeColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : : +- BroadcastExchange - : : : : : +- BroadcastHashJoin - : : : : : :- CometNativeColumnarToRow - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : : +- BroadcastExchange - : : : : : +- Project - : : : : : +- BroadcastHashJoin - : : : : : :- Project - : : : : : : +- BroadcastHashJoin - : : : : : : :- Filter - : : : : : : : +- ColumnarToRow - : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : : : +- ReusedSubquery - : : : : : : +- BroadcastExchange - : : : : : : +- CometNativeColumnarToRow - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : : +- BroadcastExchange - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- Project - : : : +- BroadcastHashJoin - : : : :- Project - : : : : +- BroadcastHashJoin - : : : : :- Filter - : : : : : +- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : +- ReusedSubquery - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- BroadcastHashJoin - : : :- CometNativeColumnarToRow - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.item - : : +- BroadcastExchange - : : +- Project - : : +- BroadcastHashJoin - : : :- CometNativeColumnarToRow - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.item - : : +- BroadcastExchange - : : +- BroadcastHashJoin - : : :- CometNativeColumnarToRow - : : : +- CometHashAggregate - : : : +- CometColumnarExchange - : : : +- HashAggregate - : : : +- Project - : : : +- BroadcastHashJoin - : : : :- Project - : : : : +- BroadcastHashJoin - : : : : :- Filter - : : : : : +- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : +- BroadcastExchange - : : : : +- BroadcastHashJoin - : : : : :- CometNativeColumnarToRow - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : +- BroadcastExchange - : : : : +- Project - : : : : +- BroadcastHashJoin - : : : : :- Project - : : : : : +- BroadcastHashJoin - : : : : : :- Filter - : : : : : : +- ColumnarToRow - : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : : +- ReusedSubquery - : : : : : +- BroadcastExchange - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- Filter - : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- ReusedSubquery - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.item - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- CometNativeColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.date_dim - :- HashAggregate - : +- CometNativeColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- HashAggregate - : +- CometNativeColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Union - : :- Filter - : : : +- Subquery - : : : +- HashAggregate - : : : +- CometNativeColumnarToRow - : : : +- CometColumnarExchange - : : : +- HashAggregate - : : : +- Union - : : : :- Project - : : : : +- BroadcastHashJoin - : : : : :- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : +- ReusedSubquery - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : :- Project - : : : : +- BroadcastHashJoin - : : : : :- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- Project - : : : +- BroadcastHashJoin - : : : :- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- ReusedSubquery - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- HashAggregate - : : +- CometNativeColumnarToRow - : : +- CometColumnarExchange - : : +- HashAggregate - : : +- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- BroadcastHashJoin - : : : : :- Filter - : : : : : +- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : +- BroadcastExchange - : : : : +- Project - : : : : +- BroadcastHashJoin - : : : : :- CometNativeColumnarToRow - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : +- BroadcastExchange - : : : : +- BroadcastHashJoin - : : : : :- CometNativeColumnarToRow - : : : : : +- CometHashAggregate - : : : : : +- CometColumnarExchange - : : : : : +- HashAggregate - : : : : : +- Project - : : : : : +- BroadcastHashJoin - : : : : : :- Project - : : : : : : +- BroadcastHashJoin - : : : : : : :- Filter - : : : : : : : +- ColumnarToRow - : : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : : : +- SubqueryBroadcast - : : : : : : : +- BroadcastExchange - : : : : : : : +- CometNativeColumnarToRow - : : : : : : : +- CometProject - : : : : : : : +- CometFilter - : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : : : +- BroadcastExchange - : : : : : : +- BroadcastHashJoin - : : : : : : :- CometNativeColumnarToRow - : : : : : : : +- CometFilter - : : : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : : : +- BroadcastExchange - : : : : : : +- Project - : : : : : : +- BroadcastHashJoin - : : : : : : :- Project - : : : : : : : +- BroadcastHashJoin - : : : : : : : :- Filter - : : : : : : : : +- ColumnarToRow - : : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : : : : +- ReusedSubquery - : : : : : : : +- BroadcastExchange - : : : : : : : +- CometNativeColumnarToRow - : : : : : : : +- CometFilter - : : : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : : : +- BroadcastExchange - : : : : : : +- CometNativeColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : : +- BroadcastExchange - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : +- BroadcastExchange - : : : : +- Project - : : : : +- BroadcastHashJoin - : : : : :- Project - : : : : : +- BroadcastHashJoin - : : : : : :- Filter - : : : : : : +- ColumnarToRow - : : : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : : +- ReusedSubquery - : : : : : +- BroadcastExchange - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- BroadcastHashJoin - : : : :- CometNativeColumnarToRow - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : +- BroadcastExchange - : : : +- Project - : : : +- BroadcastHashJoin - : : : :- CometNativeColumnarToRow - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : +- BroadcastExchange - : : : +- BroadcastHashJoin - : : : :- CometNativeColumnarToRow - : : : : +- CometHashAggregate - : : : : +- CometColumnarExchange - : : : : +- HashAggregate - : : : : +- Project - : : : : +- BroadcastHashJoin - : : : : :- Project - : : : : : +- BroadcastHashJoin - : : : : : :- Filter - : : : : : : +- ColumnarToRow - : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometNativeColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : : +- BroadcastExchange - : : : : : +- BroadcastHashJoin - : : : : : :- CometNativeColumnarToRow - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : : +- BroadcastExchange - : : : : : +- Project - : : : : : +- BroadcastHashJoin - : : : : : :- Project - : : : : : : +- BroadcastHashJoin - : : : : : : :- Filter - : : : : : : : +- ColumnarToRow - : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : : : +- ReusedSubquery - : : : : : : +- BroadcastExchange - : : : : : : +- CometNativeColumnarToRow - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : : +- BroadcastExchange - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- Project - : : : +- BroadcastHashJoin - : : : :- Project - : : : : +- BroadcastHashJoin - : : : : :- Filter - : : : : : +- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : +- ReusedSubquery - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : :- Filter - : : : +- ReusedSubquery - : : +- HashAggregate - : : +- CometNativeColumnarToRow - : : +- CometColumnarExchange - : : +- HashAggregate - : : +- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- BroadcastHashJoin - : : : : :- Filter - : : : : : +- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : +- ReusedSubquery - : : : : +- BroadcastExchange - : : : : +- Project - : : : : +- BroadcastHashJoin - : : : : :- CometNativeColumnarToRow - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : +- BroadcastExchange - : : : : +- BroadcastHashJoin - : : : : :- CometNativeColumnarToRow - : : : : : +- CometHashAggregate - : : : : : +- CometColumnarExchange - : : : : : +- HashAggregate - : : : : : +- Project - : : : : : +- BroadcastHashJoin - : : : : : :- Project - : : : : : : +- BroadcastHashJoin - : : : : : : :- Filter - : : : : : : : +- ColumnarToRow - : : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : : : +- SubqueryBroadcast - : : : : : : : +- BroadcastExchange - : : : : : : : +- CometNativeColumnarToRow - : : : : : : : +- CometProject - : : : : : : : +- CometFilter - : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : : : +- BroadcastExchange - : : : : : : +- BroadcastHashJoin - : : : : : : :- CometNativeColumnarToRow - : : : : : : : +- CometFilter - : : : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : : : +- BroadcastExchange - : : : : : : +- Project - : : : : : : +- BroadcastHashJoin - : : : : : : :- Project - : : : : : : : +- BroadcastHashJoin - : : : : : : : :- Filter - : : : : : : : : +- ColumnarToRow - : : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : : : : +- ReusedSubquery - : : : : : : : +- BroadcastExchange - : : : : : : : +- CometNativeColumnarToRow - : : : : : : : +- CometFilter - : : : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : : : +- BroadcastExchange - : : : : : : +- CometNativeColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : : +- BroadcastExchange - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : +- BroadcastExchange - : : : : +- Project - : : : : +- BroadcastHashJoin - : : : : :- Project - : : : : : +- BroadcastHashJoin - : : : : : :- Filter - : : : : : : +- ColumnarToRow - : : : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : : +- ReusedSubquery - : : : : : +- BroadcastExchange - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- BroadcastHashJoin - : : : :- CometNativeColumnarToRow - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : +- BroadcastExchange - : : : +- Project - : : : +- BroadcastHashJoin - : : : :- CometNativeColumnarToRow - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : +- BroadcastExchange - : : : +- BroadcastHashJoin - : : : :- CometNativeColumnarToRow - : : : : +- CometHashAggregate - : : : : +- CometColumnarExchange - : : : : +- HashAggregate - : : : : +- Project - : : : : +- BroadcastHashJoin - : : : : :- Project - : : : : : +- BroadcastHashJoin - : : : : : :- Filter - : : : : : : +- ColumnarToRow - : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometNativeColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : : +- BroadcastExchange - : : : : : +- BroadcastHashJoin - : : : : : :- CometNativeColumnarToRow - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : : +- BroadcastExchange - : : : : : +- Project - : : : : : +- BroadcastHashJoin - : : : : : :- Project - : : : : : : +- BroadcastHashJoin - : : : : : : :- Filter - : : : : : : : +- ColumnarToRow - : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : : : +- ReusedSubquery - : : : : : : +- BroadcastExchange - : : : : : : +- CometNativeColumnarToRow - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : : +- BroadcastExchange - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- Project - : : : +- BroadcastHashJoin - : : : :- Project - : : : : +- BroadcastHashJoin - : : : : :- Filter - : : : : : +- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : +- ReusedSubquery - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- Filter - : : +- ReusedSubquery - : +- HashAggregate - : +- CometNativeColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- BroadcastHashJoin - : : : :- Filter - : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- ReusedSubquery - : : : +- BroadcastExchange - : : : +- Project - : : : +- BroadcastHashJoin - : : : :- CometNativeColumnarToRow - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : +- BroadcastExchange - : : : +- BroadcastHashJoin - : : : :- CometNativeColumnarToRow - : : : : +- CometHashAggregate - : : : : +- CometColumnarExchange - : : : : +- HashAggregate - : : : : +- Project - : : : : +- BroadcastHashJoin - : : : : :- Project - : : : : : +- BroadcastHashJoin - : : : : : :- Filter - : : : : : : +- ColumnarToRow - : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometNativeColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : : +- BroadcastExchange - : : : : : +- BroadcastHashJoin - : : : : : :- CometNativeColumnarToRow - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : : +- BroadcastExchange - : : : : : +- Project - : : : : : +- BroadcastHashJoin - : : : : : :- Project - : : : : : : +- BroadcastHashJoin - : : : : : : :- Filter - : : : : : : : +- ColumnarToRow - : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : : : +- ReusedSubquery - : : : : : : +- BroadcastExchange - : : : : : : +- CometNativeColumnarToRow - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : : +- BroadcastExchange - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- Project - : : : +- BroadcastHashJoin - : : : :- Project - : : : : +- BroadcastHashJoin - : : : : :- Filter - : : : : : +- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : +- ReusedSubquery - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- BroadcastHashJoin - : : :- CometNativeColumnarToRow - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.item - : : +- BroadcastExchange - : : +- Project - : : +- BroadcastHashJoin - : : :- CometNativeColumnarToRow - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.item - : : +- BroadcastExchange - : : +- BroadcastHashJoin - : : :- CometNativeColumnarToRow - : : : +- CometHashAggregate - : : : +- CometColumnarExchange - : : : +- HashAggregate - : : : +- Project - : : : +- BroadcastHashJoin - : : : :- Project - : : : : +- BroadcastHashJoin - : : : : :- Filter - : : : : : +- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : +- BroadcastExchange - : : : : +- BroadcastHashJoin - : : : : :- CometNativeColumnarToRow - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : +- BroadcastExchange - : : : : +- Project - : : : : +- BroadcastHashJoin - : : : : :- Project - : : : : : +- BroadcastHashJoin - : : : : : :- Filter - : : : : : : +- ColumnarToRow - : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : : +- ReusedSubquery - : : : : : +- BroadcastExchange - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- Filter - : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- ReusedSubquery - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.item - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- CometNativeColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.date_dim - :- HashAggregate - : +- CometNativeColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- HashAggregate - : +- CometNativeColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Union - : :- Filter - : : : +- Subquery - : : : +- HashAggregate - : : : +- CometNativeColumnarToRow - : : : +- CometColumnarExchange - : : : +- HashAggregate - : : : +- Union - : : : :- Project - : : : : +- BroadcastHashJoin - : : : : :- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : +- ReusedSubquery - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : :- Project - : : : : +- BroadcastHashJoin - : : : : :- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- Project - : : : +- BroadcastHashJoin - : : : :- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- ReusedSubquery - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- HashAggregate - : : +- CometNativeColumnarToRow - : : +- CometColumnarExchange - : : +- HashAggregate - : : +- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- BroadcastHashJoin - : : : : :- Filter - : : : : : +- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : +- BroadcastExchange - : : : : +- Project - : : : : +- BroadcastHashJoin - : : : : :- CometNativeColumnarToRow - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : +- BroadcastExchange - : : : : +- BroadcastHashJoin - : : : : :- CometNativeColumnarToRow - : : : : : +- CometHashAggregate - : : : : : +- CometColumnarExchange - : : : : : +- HashAggregate - : : : : : +- Project - : : : : : +- BroadcastHashJoin - : : : : : :- Project - : : : : : : +- BroadcastHashJoin - : : : : : : :- Filter - : : : : : : : +- ColumnarToRow - : : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : : : +- SubqueryBroadcast - : : : : : : : +- BroadcastExchange - : : : : : : : +- CometNativeColumnarToRow - : : : : : : : +- CometProject - : : : : : : : +- CometFilter - : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : : : +- BroadcastExchange - : : : : : : +- BroadcastHashJoin - : : : : : : :- CometNativeColumnarToRow - : : : : : : : +- CometFilter - : : : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : : : +- BroadcastExchange - : : : : : : +- Project - : : : : : : +- BroadcastHashJoin - : : : : : : :- Project - : : : : : : : +- BroadcastHashJoin - : : : : : : : :- Filter - : : : : : : : : +- ColumnarToRow - : : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : : : : +- ReusedSubquery - : : : : : : : +- BroadcastExchange - : : : : : : : +- CometNativeColumnarToRow - : : : : : : : +- CometFilter - : : : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : : : +- BroadcastExchange - : : : : : : +- CometNativeColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : : +- BroadcastExchange - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : +- BroadcastExchange - : : : : +- Project - : : : : +- BroadcastHashJoin - : : : : :- Project - : : : : : +- BroadcastHashJoin - : : : : : :- Filter - : : : : : : +- ColumnarToRow - : : : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : : +- ReusedSubquery - : : : : : +- BroadcastExchange - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- BroadcastHashJoin - : : : :- CometNativeColumnarToRow - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : +- BroadcastExchange - : : : +- Project - : : : +- BroadcastHashJoin - : : : :- CometNativeColumnarToRow - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : +- BroadcastExchange - : : : +- BroadcastHashJoin - : : : :- CometNativeColumnarToRow - : : : : +- CometHashAggregate - : : : : +- CometColumnarExchange - : : : : +- HashAggregate - : : : : +- Project - : : : : +- BroadcastHashJoin - : : : : :- Project - : : : : : +- BroadcastHashJoin - : : : : : :- Filter - : : : : : : +- ColumnarToRow - : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometNativeColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : : +- BroadcastExchange - : : : : : +- BroadcastHashJoin - : : : : : :- CometNativeColumnarToRow - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : : +- BroadcastExchange - : : : : : +- Project - : : : : : +- BroadcastHashJoin - : : : : : :- Project - : : : : : : +- BroadcastHashJoin - : : : : : : :- Filter - : : : : : : : +- ColumnarToRow - : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : : : +- ReusedSubquery - : : : : : : +- BroadcastExchange - : : : : : : +- CometNativeColumnarToRow - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : : +- BroadcastExchange - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- Project - : : : +- BroadcastHashJoin - : : : :- Project - : : : : +- BroadcastHashJoin - : : : : :- Filter - : : : : : +- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : +- ReusedSubquery - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : :- Filter - : : : +- ReusedSubquery - : : +- HashAggregate - : : +- CometNativeColumnarToRow - : : +- CometColumnarExchange - : : +- HashAggregate - : : +- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- BroadcastHashJoin - : : : : :- Filter - : : : : : +- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : +- ReusedSubquery - : : : : +- BroadcastExchange - : : : : +- Project - : : : : +- BroadcastHashJoin - : : : : :- CometNativeColumnarToRow - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : +- BroadcastExchange - : : : : +- BroadcastHashJoin - : : : : :- CometNativeColumnarToRow - : : : : : +- CometHashAggregate - : : : : : +- CometColumnarExchange - : : : : : +- HashAggregate - : : : : : +- Project - : : : : : +- BroadcastHashJoin - : : : : : :- Project - : : : : : : +- BroadcastHashJoin - : : : : : : :- Filter - : : : : : : : +- ColumnarToRow - : : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : : : +- SubqueryBroadcast - : : : : : : : +- BroadcastExchange - : : : : : : : +- CometNativeColumnarToRow - : : : : : : : +- CometProject - : : : : : : : +- CometFilter - : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : : : +- BroadcastExchange - : : : : : : +- BroadcastHashJoin - : : : : : : :- CometNativeColumnarToRow - : : : : : : : +- CometFilter - : : : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : : : +- BroadcastExchange - : : : : : : +- Project - : : : : : : +- BroadcastHashJoin - : : : : : : :- Project - : : : : : : : +- BroadcastHashJoin - : : : : : : : :- Filter - : : : : : : : : +- ColumnarToRow - : : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : : : : +- ReusedSubquery - : : : : : : : +- BroadcastExchange - : : : : : : : +- CometNativeColumnarToRow - : : : : : : : +- CometFilter - : : : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : : : +- BroadcastExchange - : : : : : : +- CometNativeColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : : +- BroadcastExchange - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : +- BroadcastExchange - : : : : +- Project - : : : : +- BroadcastHashJoin - : : : : :- Project - : : : : : +- BroadcastHashJoin - : : : : : :- Filter - : : : : : : +- ColumnarToRow - : : : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : : +- ReusedSubquery - : : : : : +- BroadcastExchange - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- BroadcastHashJoin - : : : :- CometNativeColumnarToRow - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : +- BroadcastExchange - : : : +- Project - : : : +- BroadcastHashJoin - : : : :- CometNativeColumnarToRow - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : +- BroadcastExchange - : : : +- BroadcastHashJoin - : : : :- CometNativeColumnarToRow - : : : : +- CometHashAggregate - : : : : +- CometColumnarExchange - : : : : +- HashAggregate - : : : : +- Project - : : : : +- BroadcastHashJoin - : : : : :- Project - : : : : : +- BroadcastHashJoin - : : : : : :- Filter - : : : : : : +- ColumnarToRow - : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometNativeColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : : +- BroadcastExchange - : : : : : +- BroadcastHashJoin - : : : : : :- CometNativeColumnarToRow - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : : +- BroadcastExchange - : : : : : +- Project - : : : : : +- BroadcastHashJoin - : : : : : :- Project - : : : : : : +- BroadcastHashJoin - : : : : : : :- Filter - : : : : : : : +- ColumnarToRow - : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : : : +- ReusedSubquery - : : : : : : +- BroadcastExchange - : : : : : : +- CometNativeColumnarToRow - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : : +- BroadcastExchange - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- Project - : : : +- BroadcastHashJoin - : : : :- Project - : : : : +- BroadcastHashJoin - : : : : :- Filter - : : : : : +- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : +- ReusedSubquery - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- Filter - : : +- ReusedSubquery - : +- HashAggregate - : +- CometNativeColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- BroadcastHashJoin - : : : :- Filter - : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- ReusedSubquery - : : : +- BroadcastExchange - : : : +- Project - : : : +- BroadcastHashJoin - : : : :- CometNativeColumnarToRow - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : +- BroadcastExchange - : : : +- BroadcastHashJoin - : : : :- CometNativeColumnarToRow - : : : : +- CometHashAggregate - : : : : +- CometColumnarExchange - : : : : +- HashAggregate - : : : : +- Project - : : : : +- BroadcastHashJoin - : : : : :- Project - : : : : : +- BroadcastHashJoin - : : : : : :- Filter - : : : : : : +- ColumnarToRow - : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometNativeColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : : +- BroadcastExchange - : : : : : +- BroadcastHashJoin - : : : : : :- CometNativeColumnarToRow - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : : +- BroadcastExchange - : : : : : +- Project - : : : : : +- BroadcastHashJoin - : : : : : :- Project - : : : : : : +- BroadcastHashJoin - : : : : : : :- Filter - : : : : : : : +- ColumnarToRow - : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : : : +- ReusedSubquery - : : : : : : +- BroadcastExchange - : : : : : : +- CometNativeColumnarToRow - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : : +- BroadcastExchange - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- Project - : : : +- BroadcastHashJoin - : : : :- Project - : : : : +- BroadcastHashJoin - : : : : :- Filter - : : : : : +- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : +- ReusedSubquery - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- BroadcastHashJoin - : : :- CometNativeColumnarToRow - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.item - : : +- BroadcastExchange - : : +- Project - : : +- BroadcastHashJoin - : : :- CometNativeColumnarToRow - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.item - : : +- BroadcastExchange - : : +- BroadcastHashJoin - : : :- CometNativeColumnarToRow - : : : +- CometHashAggregate - : : : +- CometColumnarExchange - : : : +- HashAggregate - : : : +- Project - : : : +- BroadcastHashJoin - : : : :- Project - : : : : +- BroadcastHashJoin - : : : : :- Filter - : : : : : +- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : +- BroadcastExchange - : : : : +- BroadcastHashJoin - : : : : :- CometNativeColumnarToRow - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : +- BroadcastExchange - : : : : +- Project - : : : : +- BroadcastHashJoin - : : : : :- Project - : : : : : +- BroadcastHashJoin - : : : : : :- Filter - : : : : : : +- ColumnarToRow - : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : : +- ReusedSubquery - : : : : : +- BroadcastExchange - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- Filter - : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- ReusedSubquery - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.item - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- CometNativeColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.date_dim - +- HashAggregate - +- CometNativeColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- HashAggregate - +- CometNativeColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Union - :- Filter - : : +- Subquery - : : +- HashAggregate - : : +- CometNativeColumnarToRow - : : +- CometColumnarExchange - : : +- HashAggregate - : : +- Union - : : :- Project - : : : +- BroadcastHashJoin - : : : :- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- ReusedSubquery - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : :- Project - : : : +- BroadcastHashJoin - : : : :- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- Project - : : +- BroadcastHashJoin - : : :- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : +- ReusedSubquery - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- HashAggregate - : +- CometNativeColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- BroadcastHashJoin - : : : :- Filter - : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- Project - : : : +- BroadcastHashJoin - : : : :- CometNativeColumnarToRow - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : +- BroadcastExchange - : : : +- BroadcastHashJoin - : : : :- CometNativeColumnarToRow - : : : : +- CometHashAggregate - : : : : +- CometColumnarExchange - : : : : +- HashAggregate - : : : : +- Project - : : : : +- BroadcastHashJoin - : : : : :- Project - : : : : : +- BroadcastHashJoin - : : : : : :- Filter - : : : : : : +- ColumnarToRow - : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometNativeColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : : +- BroadcastExchange - : : : : : +- BroadcastHashJoin - : : : : : :- CometNativeColumnarToRow - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : : +- BroadcastExchange - : : : : : +- Project - : : : : : +- BroadcastHashJoin - : : : : : :- Project - : : : : : : +- BroadcastHashJoin - : : : : : : :- Filter - : : : : : : : +- ColumnarToRow - : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : : : +- ReusedSubquery - : : : : : : +- BroadcastExchange - : : : : : : +- CometNativeColumnarToRow - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : : +- BroadcastExchange - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- Project - : : : +- BroadcastHashJoin - : : : :- Project - : : : : +- BroadcastHashJoin - : : : : :- Filter - : : : : : +- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : +- ReusedSubquery - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- BroadcastHashJoin - : : :- CometNativeColumnarToRow - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.item - : : +- BroadcastExchange - : : +- Project - : : +- BroadcastHashJoin - : : :- CometNativeColumnarToRow - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.item - : : +- BroadcastExchange - : : +- BroadcastHashJoin - : : :- CometNativeColumnarToRow - : : : +- CometHashAggregate - : : : +- CometColumnarExchange - : : : +- HashAggregate - : : : +- Project - : : : +- BroadcastHashJoin - : : : :- Project - : : : : +- BroadcastHashJoin - : : : : :- Filter - : : : : : +- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : +- BroadcastExchange - : : : : +- BroadcastHashJoin - : : : : :- CometNativeColumnarToRow - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : +- BroadcastExchange - : : : : +- Project - : : : : +- BroadcastHashJoin - : : : : :- Project - : : : : : +- BroadcastHashJoin - : : : : : :- Filter - : : : : : : +- ColumnarToRow - : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : : +- ReusedSubquery - : : : : : +- BroadcastExchange - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- Filter - : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- ReusedSubquery - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.item - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- CometNativeColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.date_dim - :- Filter - : : +- ReusedSubquery - : +- HashAggregate - : +- CometNativeColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- BroadcastHashJoin - : : : :- Filter - : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- ReusedSubquery - : : : +- BroadcastExchange - : : : +- Project - : : : +- BroadcastHashJoin - : : : :- CometNativeColumnarToRow - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : +- BroadcastExchange - : : : +- BroadcastHashJoin - : : : :- CometNativeColumnarToRow - : : : : +- CometHashAggregate - : : : : +- CometColumnarExchange - : : : : +- HashAggregate - : : : : +- Project - : : : : +- BroadcastHashJoin - : : : : :- Project - : : : : : +- BroadcastHashJoin - : : : : : :- Filter - : : : : : : +- ColumnarToRow - : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometNativeColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : : +- BroadcastExchange - : : : : : +- BroadcastHashJoin - : : : : : :- CometNativeColumnarToRow - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : : +- BroadcastExchange - : : : : : +- Project - : : : : : +- BroadcastHashJoin - : : : : : :- Project - : : : : : : +- BroadcastHashJoin - : : : : : : :- Filter - : : : : : : : +- ColumnarToRow - : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : : : +- ReusedSubquery - : : : : : : +- BroadcastExchange - : : : : : : +- CometNativeColumnarToRow - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : : +- BroadcastExchange - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- Project - : : : +- BroadcastHashJoin - : : : :- Project - : : : : +- BroadcastHashJoin - : : : : :- Filter - : : : : : +- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : +- ReusedSubquery - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- BroadcastHashJoin - : : :- CometNativeColumnarToRow - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.item - : : +- BroadcastExchange - : : +- Project - : : +- BroadcastHashJoin - : : :- CometNativeColumnarToRow - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.item - : : +- BroadcastExchange - : : +- BroadcastHashJoin - : : :- CometNativeColumnarToRow - : : : +- CometHashAggregate - : : : +- CometColumnarExchange - : : : +- HashAggregate - : : : +- Project - : : : +- BroadcastHashJoin - : : : :- Project - : : : : +- BroadcastHashJoin - : : : : :- Filter - : : : : : +- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : +- BroadcastExchange - : : : : +- BroadcastHashJoin - : : : : :- CometNativeColumnarToRow - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : +- BroadcastExchange - : : : : +- Project - : : : : +- BroadcastHashJoin - : : : : :- Project - : : : : : +- BroadcastHashJoin - : : : : : :- Filter - : : : : : : +- ColumnarToRow - : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : : +- ReusedSubquery - : : : : : +- BroadcastExchange - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- Filter - : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- ReusedSubquery - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.item - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- CometNativeColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.date_dim - +- Filter - : +- ReusedSubquery - +- HashAggregate - +- CometNativeColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- BroadcastHashJoin - : : :- Filter - : : : +- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : +- ReusedSubquery - : : +- BroadcastExchange - : : +- Project - : : +- BroadcastHashJoin - : : :- CometNativeColumnarToRow - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.item - : : +- BroadcastExchange - : : +- BroadcastHashJoin - : : :- CometNativeColumnarToRow - : : : +- CometHashAggregate - : : : +- CometColumnarExchange - : : : +- HashAggregate - : : : +- Project - : : : +- BroadcastHashJoin - : : : :- Project - : : : : +- BroadcastHashJoin - : : : : :- Filter - : : : : : +- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : +- BroadcastExchange - : : : : +- BroadcastHashJoin - : : : : :- CometNativeColumnarToRow - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : +- BroadcastExchange - : : : : +- Project - : : : : +- BroadcastHashJoin - : : : : :- Project - : : : : : +- BroadcastHashJoin - : : : : : :- Filter - : : : : : : +- ColumnarToRow - : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : : +- ReusedSubquery - : : : : : +- BroadcastExchange - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- Filter - : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- ReusedSubquery - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.item - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- BroadcastHashJoin - : :- CometNativeColumnarToRow - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.item - : +- BroadcastExchange - : +- Project - : +- BroadcastHashJoin - : :- CometNativeColumnarToRow - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.item - : +- BroadcastExchange - : +- BroadcastHashJoin - : :- CometNativeColumnarToRow - : : +- CometHashAggregate - : : +- CometColumnarExchange - : : +- HashAggregate - : : +- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- Filter - : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- BroadcastHashJoin - : : : :- CometNativeColumnarToRow - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : +- BroadcastExchange - : : : +- Project - : : : +- BroadcastHashJoin - : : : :- Project - : : : : +- BroadcastHashJoin - : : : : :- Filter - : : : : : +- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : +- ReusedSubquery - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Filter - : : : +- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : +- ReusedSubquery - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.item - : +- BroadcastExchange - : +- CometNativeColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.date_dim - +- BroadcastExchange - +- CometNativeColumnarToRow - +- CometProject - +- CometFilter - +- CometNativeScan parquet spark_catalog.default.date_dim + :- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometUnion + : :- CometFilter + : : : +- Subquery + : : : +- CometNativeColumnarToRow + : : : +- CometHashAggregate + : : : +- CometExchange + : : : +- CometHashAggregate + : : : +- CometUnion + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometNativeScan parquet spark_catalog.default.store_sales + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometNativeColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometNativeColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometNativeScan parquet spark_catalog.default.web_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometNativeColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometHashAggregate + : : +- CometExchange + : : +- CometHashAggregate + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometBroadcastHashJoin + : : : : :- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometNativeColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : : +- CometBroadcastExchange + : : : : +- CometBroadcastHashJoin + : : : : :- CometHashAggregate + : : : : : +- CometExchange + : : : : : +- CometHashAggregate + : : : : : +- CometProject + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometProject + : : : : : : +- CometBroadcastHashJoin + : : : : : : :- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : : : : +- SubqueryBroadcast + : : : : : : : +- BroadcastExchange + : : : : : : : +- CometNativeColumnarToRow + : : : : : : : +- CometProject + : : : : : : : +- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : +- CometBroadcastExchange + : : : : : : +- CometBroadcastHashJoin + : : : : : : :- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : : : : +- CometBroadcastExchange + : : : : : : +- CometProject + : : : : : : +- CometBroadcastHashJoin + : : : : : : :- CometProject + : : : : : : : +- CometBroadcastHashJoin + : : : : : : : :- CometFilter + : : : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : : : : : +- SubqueryBroadcast + : : : : : : : : +- BroadcastExchange + : : : : : : : : +- CometNativeColumnarToRow + : : : : : : : : +- CometProject + : : : : : : : : +- CometFilter + : : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : : +- CometBroadcastExchange + : : : : : : : +- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : : : : +- CometBroadcastExchange + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- CometBroadcastExchange + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometProject + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.web_sales + : : : : : : +- SubqueryBroadcast + : : : : : : +- BroadcastExchange + : : : : : : +- CometNativeColumnarToRow + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- CometBroadcastExchange + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : +- CometBroadcastExchange + : : : +- CometBroadcastHashJoin + : : : :- CometHashAggregate + : : : : +- CometExchange + : : : : +- CometHashAggregate + : : : : +- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometProject + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : : : +- SubqueryBroadcast + : : : : : : +- BroadcastExchange + : : : : : : +- CometNativeColumnarToRow + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- CometBroadcastExchange + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : : : +- CometBroadcastExchange + : : : : : +- CometProject + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometProject + : : : : : : +- CometBroadcastHashJoin + : : : : : : :- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : : : : +- SubqueryBroadcast + : : : : : : : +- BroadcastExchange + : : : : : : : +- CometNativeColumnarToRow + : : : : : : : +- CometProject + : : : : : : : +- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : +- CometBroadcastExchange + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : : : +- CometBroadcastExchange + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.web_sales + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometNativeColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : :- CometFilter + : : : +- ReusedSubquery + : : +- CometHashAggregate + : : +- CometExchange + : : +- CometHashAggregate + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometBroadcastHashJoin + : : : : :- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometNativeColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : : +- CometBroadcastExchange + : : : : +- CometBroadcastHashJoin + : : : : :- CometHashAggregate + : : : : : +- CometExchange + : : : : : +- CometHashAggregate + : : : : : +- CometProject + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometProject + : : : : : : +- CometBroadcastHashJoin + : : : : : : :- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : : : : +- SubqueryBroadcast + : : : : : : : +- BroadcastExchange + : : : : : : : +- CometNativeColumnarToRow + : : : : : : : +- CometProject + : : : : : : : +- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : +- CometBroadcastExchange + : : : : : : +- CometBroadcastHashJoin + : : : : : : :- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : : : : +- CometBroadcastExchange + : : : : : : +- CometProject + : : : : : : +- CometBroadcastHashJoin + : : : : : : :- CometProject + : : : : : : : +- CometBroadcastHashJoin + : : : : : : : :- CometFilter + : : : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : : : : : +- SubqueryBroadcast + : : : : : : : : +- BroadcastExchange + : : : : : : : : +- CometNativeColumnarToRow + : : : : : : : : +- CometProject + : : : : : : : : +- CometFilter + : : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : : +- CometBroadcastExchange + : : : : : : : +- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : : : : +- CometBroadcastExchange + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- CometBroadcastExchange + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometProject + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.web_sales + : : : : : : +- SubqueryBroadcast + : : : : : : +- BroadcastExchange + : : : : : : +- CometNativeColumnarToRow + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- CometBroadcastExchange + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : +- CometBroadcastExchange + : : : +- CometBroadcastHashJoin + : : : :- CometHashAggregate + : : : : +- CometExchange + : : : : +- CometHashAggregate + : : : : +- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometProject + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : : : +- SubqueryBroadcast + : : : : : : +- BroadcastExchange + : : : : : : +- CometNativeColumnarToRow + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- CometBroadcastExchange + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : : : +- CometBroadcastExchange + : : : : : +- CometProject + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometProject + : : : : : : +- CometBroadcastHashJoin + : : : : : : :- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : : : : +- SubqueryBroadcast + : : : : : : : +- BroadcastExchange + : : : : : : : +- CometNativeColumnarToRow + : : : : : : : +- CometProject + : : : : : : : +- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : +- CometBroadcastExchange + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : : : +- CometBroadcastExchange + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.web_sales + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometNativeColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometFilter + : : +- ReusedSubquery + : +- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.web_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometNativeColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : +- CometBroadcastExchange + : : : +- CometBroadcastHashJoin + : : : :- CometHashAggregate + : : : : +- CometExchange + : : : : +- CometHashAggregate + : : : : +- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometProject + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : : : +- SubqueryBroadcast + : : : : : : +- BroadcastExchange + : : : : : : +- CometNativeColumnarToRow + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- CometBroadcastExchange + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : : : +- CometBroadcastExchange + : : : : : +- CometProject + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometProject + : : : : : : +- CometBroadcastHashJoin + : : : : : : :- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : : : : +- SubqueryBroadcast + : : : : : : : +- BroadcastExchange + : : : : : : : +- CometNativeColumnarToRow + : : : : : : : +- CometProject + : : : : : : : +- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : +- CometBroadcastExchange + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : : : +- CometBroadcastExchange + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.web_sales + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometNativeColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.item + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.item + : : +- CometBroadcastExchange + : : +- CometBroadcastHashJoin + : : :- CometHashAggregate + : : : +- CometExchange + : : : +- CometHashAggregate + : : : +- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometNativeColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometBroadcastHashJoin + : : : : :- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometProject + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : : : +- SubqueryBroadcast + : : : : : : +- BroadcastExchange + : : : : : : +- CometNativeColumnarToRow + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- CometBroadcastExchange + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.web_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometNativeColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.item + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.date_dim + :- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometUnion + : :- CometFilter + : : : +- Subquery + : : : +- CometNativeColumnarToRow + : : : +- CometHashAggregate + : : : +- CometExchange + : : : +- CometHashAggregate + : : : +- CometUnion + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometNativeScan parquet spark_catalog.default.store_sales + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometNativeColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometNativeColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometNativeScan parquet spark_catalog.default.web_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometNativeColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometHashAggregate + : : +- CometExchange + : : +- CometHashAggregate + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometBroadcastHashJoin + : : : : :- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometNativeColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : : +- CometBroadcastExchange + : : : : +- CometBroadcastHashJoin + : : : : :- CometHashAggregate + : : : : : +- CometExchange + : : : : : +- CometHashAggregate + : : : : : +- CometProject + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometProject + : : : : : : +- CometBroadcastHashJoin + : : : : : : :- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : : : : +- SubqueryBroadcast + : : : : : : : +- BroadcastExchange + : : : : : : : +- CometNativeColumnarToRow + : : : : : : : +- CometProject + : : : : : : : +- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : +- CometBroadcastExchange + : : : : : : +- CometBroadcastHashJoin + : : : : : : :- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : : : : +- CometBroadcastExchange + : : : : : : +- CometProject + : : : : : : +- CometBroadcastHashJoin + : : : : : : :- CometProject + : : : : : : : +- CometBroadcastHashJoin + : : : : : : : :- CometFilter + : : : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : : : : : +- SubqueryBroadcast + : : : : : : : : +- BroadcastExchange + : : : : : : : : +- CometNativeColumnarToRow + : : : : : : : : +- CometProject + : : : : : : : : +- CometFilter + : : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : : +- CometBroadcastExchange + : : : : : : : +- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : : : : +- CometBroadcastExchange + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- CometBroadcastExchange + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometProject + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.web_sales + : : : : : : +- SubqueryBroadcast + : : : : : : +- BroadcastExchange + : : : : : : +- CometNativeColumnarToRow + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- CometBroadcastExchange + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : +- CometBroadcastExchange + : : : +- CometBroadcastHashJoin + : : : :- CometHashAggregate + : : : : +- CometExchange + : : : : +- CometHashAggregate + : : : : +- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometProject + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : : : +- SubqueryBroadcast + : : : : : : +- BroadcastExchange + : : : : : : +- CometNativeColumnarToRow + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- CometBroadcastExchange + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : : : +- CometBroadcastExchange + : : : : : +- CometProject + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometProject + : : : : : : +- CometBroadcastHashJoin + : : : : : : :- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : : : : +- SubqueryBroadcast + : : : : : : : +- BroadcastExchange + : : : : : : : +- CometNativeColumnarToRow + : : : : : : : +- CometProject + : : : : : : : +- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : +- CometBroadcastExchange + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : : : +- CometBroadcastExchange + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.web_sales + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometNativeColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : :- CometFilter + : : : +- ReusedSubquery + : : +- CometHashAggregate + : : +- CometExchange + : : +- CometHashAggregate + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometBroadcastHashJoin + : : : : :- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometNativeColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : : +- CometBroadcastExchange + : : : : +- CometBroadcastHashJoin + : : : : :- CometHashAggregate + : : : : : +- CometExchange + : : : : : +- CometHashAggregate + : : : : : +- CometProject + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometProject + : : : : : : +- CometBroadcastHashJoin + : : : : : : :- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : : : : +- SubqueryBroadcast + : : : : : : : +- BroadcastExchange + : : : : : : : +- CometNativeColumnarToRow + : : : : : : : +- CometProject + : : : : : : : +- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : +- CometBroadcastExchange + : : : : : : +- CometBroadcastHashJoin + : : : : : : :- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : : : : +- CometBroadcastExchange + : : : : : : +- CometProject + : : : : : : +- CometBroadcastHashJoin + : : : : : : :- CometProject + : : : : : : : +- CometBroadcastHashJoin + : : : : : : : :- CometFilter + : : : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : : : : : +- SubqueryBroadcast + : : : : : : : : +- BroadcastExchange + : : : : : : : : +- CometNativeColumnarToRow + : : : : : : : : +- CometProject + : : : : : : : : +- CometFilter + : : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : : +- CometBroadcastExchange + : : : : : : : +- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : : : : +- CometBroadcastExchange + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- CometBroadcastExchange + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometProject + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.web_sales + : : : : : : +- SubqueryBroadcast + : : : : : : +- BroadcastExchange + : : : : : : +- CometNativeColumnarToRow + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- CometBroadcastExchange + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : +- CometBroadcastExchange + : : : +- CometBroadcastHashJoin + : : : :- CometHashAggregate + : : : : +- CometExchange + : : : : +- CometHashAggregate + : : : : +- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometProject + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : : : +- SubqueryBroadcast + : : : : : : +- BroadcastExchange + : : : : : : +- CometNativeColumnarToRow + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- CometBroadcastExchange + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : : : +- CometBroadcastExchange + : : : : : +- CometProject + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometProject + : : : : : : +- CometBroadcastHashJoin + : : : : : : :- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : : : : +- SubqueryBroadcast + : : : : : : : +- BroadcastExchange + : : : : : : : +- CometNativeColumnarToRow + : : : : : : : +- CometProject + : : : : : : : +- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : +- CometBroadcastExchange + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : : : +- CometBroadcastExchange + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.web_sales + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometNativeColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometFilter + : : +- ReusedSubquery + : +- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.web_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometNativeColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : +- CometBroadcastExchange + : : : +- CometBroadcastHashJoin + : : : :- CometHashAggregate + : : : : +- CometExchange + : : : : +- CometHashAggregate + : : : : +- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometProject + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : : : +- SubqueryBroadcast + : : : : : : +- BroadcastExchange + : : : : : : +- CometNativeColumnarToRow + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- CometBroadcastExchange + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : : : +- CometBroadcastExchange + : : : : : +- CometProject + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometProject + : : : : : : +- CometBroadcastHashJoin + : : : : : : :- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : : : : +- SubqueryBroadcast + : : : : : : : +- BroadcastExchange + : : : : : : : +- CometNativeColumnarToRow + : : : : : : : +- CometProject + : : : : : : : +- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : +- CometBroadcastExchange + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : : : +- CometBroadcastExchange + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.web_sales + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometNativeColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.item + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.item + : : +- CometBroadcastExchange + : : +- CometBroadcastHashJoin + : : :- CometHashAggregate + : : : +- CometExchange + : : : +- CometHashAggregate + : : : +- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometNativeColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometBroadcastHashJoin + : : : : :- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometProject + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : : : +- SubqueryBroadcast + : : : : : : +- BroadcastExchange + : : : : : : +- CometNativeColumnarToRow + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- CometBroadcastExchange + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.web_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometNativeColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.item + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.date_dim + +- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometUnion + :- CometFilter + : : +- Subquery + : : +- CometNativeColumnarToRow + : : +- CometHashAggregate + : : +- CometExchange + : : +- CometHashAggregate + : : +- CometUnion + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometNativeScan parquet spark_catalog.default.store_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometNativeColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometNativeColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometNativeScan parquet spark_catalog.default.web_sales + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometNativeColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometNativeColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : +- CometBroadcastExchange + : : : +- CometBroadcastHashJoin + : : : :- CometHashAggregate + : : : : +- CometExchange + : : : : +- CometHashAggregate + : : : : +- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometProject + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : : : +- SubqueryBroadcast + : : : : : : +- BroadcastExchange + : : : : : : +- CometNativeColumnarToRow + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- CometBroadcastExchange + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : : : +- CometBroadcastExchange + : : : : : +- CometProject + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometProject + : : : : : : +- CometBroadcastHashJoin + : : : : : : :- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : : : : +- SubqueryBroadcast + : : : : : : : +- BroadcastExchange + : : : : : : : +- CometNativeColumnarToRow + : : : : : : : +- CometProject + : : : : : : : +- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : +- CometBroadcastExchange + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : : : +- CometBroadcastExchange + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.web_sales + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometNativeColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.item + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.item + : : +- CometBroadcastExchange + : : +- CometBroadcastHashJoin + : : :- CometHashAggregate + : : : +- CometExchange + : : : +- CometHashAggregate + : : : +- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometNativeColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometBroadcastHashJoin + : : : : :- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometProject + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : : : +- SubqueryBroadcast + : : : : : : +- BroadcastExchange + : : : : : : +- CometNativeColumnarToRow + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- CometBroadcastExchange + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.web_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometNativeColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.item + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.date_dim + :- CometFilter + : : +- ReusedSubquery + : +- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometNativeColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : +- CometBroadcastExchange + : : : +- CometBroadcastHashJoin + : : : :- CometHashAggregate + : : : : +- CometExchange + : : : : +- CometHashAggregate + : : : : +- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometProject + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : : : +- SubqueryBroadcast + : : : : : : +- BroadcastExchange + : : : : : : +- CometNativeColumnarToRow + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- CometBroadcastExchange + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : : : +- CometBroadcastExchange + : : : : : +- CometProject + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometProject + : : : : : : +- CometBroadcastHashJoin + : : : : : : :- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : : : : +- SubqueryBroadcast + : : : : : : : +- BroadcastExchange + : : : : : : : +- CometNativeColumnarToRow + : : : : : : : +- CometProject + : : : : : : : +- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : +- CometBroadcastExchange + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : : : +- CometBroadcastExchange + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.web_sales + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometNativeColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.item + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.item + : : +- CometBroadcastExchange + : : +- CometBroadcastHashJoin + : : :- CometHashAggregate + : : : +- CometExchange + : : : +- CometHashAggregate + : : : +- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometNativeColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometBroadcastHashJoin + : : : : :- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometProject + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : : : +- SubqueryBroadcast + : : : : : : +- BroadcastExchange + : : : : : : +- CometNativeColumnarToRow + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- CometBroadcastExchange + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.web_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometNativeColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.item + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.date_dim + +- CometFilter + : +- ReusedSubquery + +- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.web_sales + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometNativeColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.item + : : +- CometBroadcastExchange + : : +- CometBroadcastHashJoin + : : :- CometHashAggregate + : : : +- CometExchange + : : : +- CometHashAggregate + : : : +- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometNativeColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometBroadcastHashJoin + : : : : :- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometProject + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : : : +- SubqueryBroadcast + : : : : : : +- BroadcastExchange + : : : : : : +- CometNativeColumnarToRow + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- CometBroadcastExchange + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.web_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometNativeColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.item + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometBroadcastHashJoin + : :- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.item + : +- CometBroadcastExchange + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.item + : +- CometBroadcastExchange + : +- CometBroadcastHashJoin + : :- CometHashAggregate + : : +- CometExchange + : : +- CometHashAggregate + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometNativeColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometNativeColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.web_sales + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometNativeColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.item + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.date_dim + +- CometBroadcastExchange + +- CometProject + +- CometFilter + +- CometNativeScan parquet spark_catalog.default.date_dim -Comet accelerated 842 out of 2302 eligible operators (36%). Final plan contains 475 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 2367 out of 2622 eligible operators (90%). Final plan contains 126 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q18a.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q18a.native_datafusion/extended.txt index 8ad4e890ec..82bacea2fb 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q18a.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q18a.native_datafusion/extended.txt @@ -1,253 +1,234 @@ -TakeOrderedAndProject -+- Union - :- HashAggregate - : +- CometNativeColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- Project - : : : : +- BroadcastHashJoin - : : : : :- Project - : : : : : +- BroadcastHashJoin - : : : : : :- Project - : : : : : : +- BroadcastHashJoin - : : : : : : :- Filter - : : : : : : : +- ColumnarToRow - : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : : : +- SubqueryBroadcast - : : : : : : : +- BroadcastExchange - : : : : : : : +- CometNativeColumnarToRow - : : : : : : : +- CometProject - : : : : : : : +- CometFilter - : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : : : +- BroadcastExchange - : : : : : : +- CometNativeColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.customer_demographics - : : : : : +- BroadcastExchange - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.customer - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.customer_demographics - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.customer_address - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- CometNativeColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.item - :- HashAggregate - : +- CometNativeColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- Project - : : : : +- BroadcastHashJoin - : : : : :- Project - : : : : : +- BroadcastHashJoin - : : : : : :- Project - : : : : : : +- BroadcastHashJoin - : : : : : : :- Filter - : : : : : : : +- ColumnarToRow - : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : : : +- ReusedSubquery - : : : : : : +- BroadcastExchange - : : : : : : +- CometNativeColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.customer_demographics - : : : : : +- BroadcastExchange - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.customer - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.customer_demographics - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.customer_address - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- CometNativeColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.item - :- HashAggregate - : +- CometNativeColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- Project - : : : : +- BroadcastHashJoin - : : : : :- Project - : : : : : +- BroadcastHashJoin - : : : : : :- Project - : : : : : : +- BroadcastHashJoin - : : : : : : :- Filter - : : : : : : : +- ColumnarToRow - : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : : : +- ReusedSubquery - : : : : : : +- BroadcastExchange - : : : : : : +- CometNativeColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.customer_demographics - : : : : : +- BroadcastExchange - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.customer - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.customer_demographics - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.customer_address - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- CometNativeColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.item - :- HashAggregate - : +- CometNativeColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- Project - : : : : +- BroadcastHashJoin - : : : : :- Project - : : : : : +- BroadcastHashJoin - : : : : : :- Project - : : : : : : +- BroadcastHashJoin - : : : : : : :- Filter - : : : : : : : +- ColumnarToRow - : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : : : +- ReusedSubquery - : : : : : : +- BroadcastExchange - : : : : : : +- CometNativeColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.customer_demographics - : : : : : +- BroadcastExchange - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.customer - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.customer_demographics - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.customer_address - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- CometNativeColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.item - +- HashAggregate - +- CometNativeColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- Project - : : : : +- BroadcastHashJoin - : : : : :- Project - : : : : : +- BroadcastHashJoin - : : : : : :- Filter - : : : : : : +- ColumnarToRow - : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : : +- ReusedSubquery - : : : : : +- BroadcastExchange - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.customer_demographics - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.customer - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.customer_demographics - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.customer_address - : +- BroadcastExchange - : +- CometNativeColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.date_dim - +- BroadcastExchange - +- CometNativeColumnarToRow - +- CometFilter - +- CometNativeScan parquet spark_catalog.default.item +CometNativeColumnarToRow ++- CometTakeOrderedAndProject + +- CometUnion + :- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometProject + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometProject + : : : : : : +- CometBroadcastHashJoin + : : : : : : :- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : : : : +- SubqueryBroadcast + : : : : : : : +- BroadcastExchange + : : : : : : : +- CometNativeColumnarToRow + : : : : : : : +- CometProject + : : : : : : : +- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : +- CometBroadcastExchange + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.customer_demographics + : : : : : +- CometBroadcastExchange + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.customer + : : : : +- CometBroadcastExchange + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.customer_demographics + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.customer_address + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.item + :- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometProject + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometProject + : : : : : : +- CometBroadcastHashJoin + : : : : : : :- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : : : : +- SubqueryBroadcast + : : : : : : : +- BroadcastExchange + : : : : : : : +- CometNativeColumnarToRow + : : : : : : : +- CometProject + : : : : : : : +- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : +- CometBroadcastExchange + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.customer_demographics + : : : : : +- CometBroadcastExchange + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.customer + : : : : +- CometBroadcastExchange + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.customer_demographics + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.customer_address + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.item + :- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometProject + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometProject + : : : : : : +- CometBroadcastHashJoin + : : : : : : :- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : : : : +- SubqueryBroadcast + : : : : : : : +- BroadcastExchange + : : : : : : : +- CometNativeColumnarToRow + : : : : : : : +- CometProject + : : : : : : : +- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : +- CometBroadcastExchange + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.customer_demographics + : : : : : +- CometBroadcastExchange + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.customer + : : : : +- CometBroadcastExchange + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.customer_demographics + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.customer_address + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.item + :- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometProject + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometProject + : : : : : : +- CometBroadcastHashJoin + : : : : : : :- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : : : : +- SubqueryBroadcast + : : : : : : : +- BroadcastExchange + : : : : : : : +- CometNativeColumnarToRow + : : : : : : : +- CometProject + : : : : : : : +- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : +- CometBroadcastExchange + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.customer_demographics + : : : : : +- CometBroadcastExchange + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.customer + : : : : +- CometBroadcastExchange + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.customer_demographics + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.customer_address + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.item + +- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometProject + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : : : +- SubqueryBroadcast + : : : : : : +- BroadcastExchange + : : : : : : +- CometNativeColumnarToRow + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- CometBroadcastExchange + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.customer_demographics + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.customer + : : : +- CometBroadcastExchange + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.customer_demographics + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.customer_address + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.date_dim + +- CometBroadcastExchange + +- CometFilter + +- CometNativeScan parquet spark_catalog.default.item -Comet accelerated 92 out of 210 eligible operators (43%). Final plan contains 41 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 216 out of 226 eligible operators (95%). Final plan contains 6 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q20.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q20.native_datafusion/extended.txt index 6634edcbed..c4f54f4289 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q20.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q20.native_datafusion/extended.txt @@ -3,33 +3,29 @@ TakeOrderedAndProject +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] +- CometNativeColumnarToRow +- CometSort - +- CometColumnarExchange - +- HashAggregate - +- CometNativeColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- Filter - : : +- ColumnarToRow - : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- CometNativeColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.item - +- BroadcastExchange - +- CometNativeColumnarToRow - +- CometProject - +- CometFilter - +- CometNativeScan parquet spark_catalog.default.date_dim + +- CometExchange + +- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : +- SubqueryBroadcast + : : +- BroadcastExchange + : : +- CometNativeColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.item + +- CometBroadcastExchange + +- CometProject + +- CometFilter + +- CometNativeScan parquet spark_catalog.default.date_dim -Comet accelerated 12 out of 27 eligible operators (44%). Final plan contains 6 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 22 out of 27 eligible operators (81%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q22.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q22.native_datafusion/extended.txt index 108e3c13f9..a8588fd031 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q22.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q22.native_datafusion/extended.txt @@ -5,27 +5,25 @@ TakeOrderedAndProject +- HashAggregate +- Expand +- Project - +- BroadcastNestedLoopJoin - :- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Filter - : : : +- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.inventory [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- CometNativeColumnarToRow + +- BroadcastNestedLoopJoin [COMET: BroadcastNestedLoopJoin is not supported] + :- CometNativeColumnarToRow + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.inventory + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometNativeColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange : +- CometProject : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.item @@ -33,4 +31,4 @@ TakeOrderedAndProject +- CometNativeColumnarToRow +- CometNativeScan parquet spark_catalog.default.warehouse -Comet accelerated 11 out of 28 eligible operators (39%). Final plan contains 6 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 19 out of 28 eligible operators (67%). Final plan contains 4 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q22a.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q22a.native_datafusion/extended.txt index eaa5438d91..5bd5daea84 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q22a.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q22a.native_datafusion/extended.txt @@ -1,187 +1,159 @@ -TakeOrderedAndProject -+- Union - :- HashAggregate - : +- HashAggregate - : +- HashAggregate - : +- CometNativeColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- Filter - : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.inventory [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.item - : +- BroadcastExchange - : +- CometNativeColumnarToRow - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.warehouse - :- HashAggregate - : +- CometNativeColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- HashAggregate - : +- CometNativeColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- Filter - : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.inventory [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.item - : +- BroadcastExchange - : +- CometNativeColumnarToRow - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.warehouse - :- HashAggregate - : +- CometNativeColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- HashAggregate - : +- CometNativeColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- Filter - : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.inventory [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.item - : +- BroadcastExchange - : +- CometNativeColumnarToRow - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.warehouse - :- HashAggregate - : +- CometNativeColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- HashAggregate - : +- CometNativeColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- Filter - : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.inventory [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.item - : +- BroadcastExchange - : +- CometNativeColumnarToRow - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.warehouse - +- HashAggregate - +- CometNativeColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- HashAggregate - +- CometNativeColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Filter - : : : +- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.inventory [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- CometNativeColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.item - +- BroadcastExchange - +- CometNativeColumnarToRow - +- CometFilter - +- CometNativeScan parquet spark_catalog.default.warehouse +CometNativeColumnarToRow ++- CometTakeOrderedAndProject + +- CometUnion + :- CometHashAggregate + : +- CometHashAggregate + : +- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.inventory + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometNativeColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.item + : +- CometBroadcastExchange + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.warehouse + :- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.inventory + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometNativeColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.item + : +- CometBroadcastExchange + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.warehouse + :- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.inventory + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometNativeColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.item + : +- CometBroadcastExchange + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.warehouse + :- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.inventory + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometNativeColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.item + : +- CometBroadcastExchange + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.warehouse + +- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.inventory + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometNativeColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.item + +- CometBroadcastExchange + +- CometFilter + +- CometNativeScan parquet spark_catalog.default.warehouse -Comet accelerated 64 out of 151 eligible operators (42%). Final plan contains 34 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 141 out of 151 eligible operators (93%). Final plan contains 6 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q27a.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q27a.native_datafusion/extended.txt index 4eec8b0953..5dc06ab86f 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q27a.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q27a.native_datafusion/extended.txt @@ -1,116 +1,109 @@ -TakeOrderedAndProject -+- Union - :- HashAggregate - : +- CometNativeColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- Project - : : : : +- BroadcastHashJoin - : : : : :- Filter - : : : : : +- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.customer_demographics - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.store - : +- BroadcastExchange - : +- CometNativeColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.item - :- HashAggregate - : +- CometNativeColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- Project - : : : : +- BroadcastHashJoin - : : : : :- Filter - : : : : : +- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : +- ReusedSubquery - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.customer_demographics - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.store - : +- BroadcastExchange - : +- CometNativeColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.item - +- HashAggregate - +- CometNativeColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- Filter - : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- ReusedSubquery - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.customer_demographics - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- CometNativeColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.store - +- BroadcastExchange - +- CometNativeColumnarToRow - +- CometFilter - +- CometNativeScan parquet spark_catalog.default.item +CometNativeColumnarToRow ++- CometTakeOrderedAndProject + +- CometUnion + :- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometNativeColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.customer_demographics + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.store + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.item + :- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometNativeColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.customer_demographics + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.store + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.item + +- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometNativeColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.customer_demographics + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.store + +- CometBroadcastExchange + +- CometFilter + +- CometNativeScan parquet spark_catalog.default.item -Comet accelerated 41 out of 95 eligible operators (43%). Final plan contains 19 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 97 out of 103 eligible operators (94%). Final plan contains 4 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q34.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q34.native_datafusion/extended.txt index 40230582bb..1b61be2937 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q34.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q34.native_datafusion/extended.txt @@ -1,47 +1,41 @@ CometNativeColumnarToRow +- CometSort - +- CometColumnarExchange - +- Project - +- BroadcastHashJoin - :- Filter - : +- HashAggregate - : +- CometNativeColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- Filter - : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.store - : +- BroadcastExchange - : +- CometNativeColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.household_demographics - +- BroadcastExchange - +- CometNativeColumnarToRow - +- CometProject - +- CometFilter - +- CometNativeScan parquet spark_catalog.default.customer + +- CometExchange + +- CometProject + +- CometBroadcastHashJoin + :- CometFilter + : +- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometNativeColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.store + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.household_demographics + +- CometBroadcastExchange + +- CometProject + +- CometFilter + +- CometNativeScan parquet spark_catalog.default.customer -Comet accelerated 18 out of 37 eligible operators (48%). Final plan contains 8 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 35 out of 37 eligible operators (94%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q35.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q35.native_datafusion/extended.txt index 40a8ef4a6e..2785c3d0b1 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q35.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q35.native_datafusion/extended.txt @@ -10,46 +10,52 @@ TakeOrderedAndProject : :- Project : : +- Filter : : +- BroadcastHashJoin - : : :- BroadcastHashJoin - : : : :- BroadcastHashJoin - : : : : :- CometNativeColumnarToRow - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.customer - : : : : +- BroadcastExchange - : : : : +- Project - : : : : +- BroadcastHashJoin - : : : : :- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow + : : :- BroadcastHashJoin [COMET: Unsupported join type ExistenceJoin(exists#1)] + : : : :- CometNativeColumnarToRow + : : : : +- CometBroadcastHashJoin + : : : : :- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.customer + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometNativeScan parquet spark_catalog.default.store_sales + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometNativeColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange : : : : +- CometProject : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- BroadcastExchange - : : : +- Project - : : : +- BroadcastHashJoin - : : : :- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- ReusedSubquery - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow + : : : +- CometNativeColumnarToRow + : : : +- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometNativeScan parquet spark_catalog.default.web_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometNativeColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- BroadcastExchange - : : +- Project - : : +- BroadcastHashJoin - : : :- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : +- ReusedSubquery - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow + : : +- CometNativeColumnarToRow + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometNativeColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange : : +- CometProject : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -64,4 +70,4 @@ TakeOrderedAndProject +- CometFilter +- CometNativeScan parquet spark_catalog.default.customer_demographics -Comet accelerated 21 out of 54 eligible operators (38%). Final plan contains 11 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 41 out of 62 eligible operators (66%). Final plan contains 9 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q35a.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q35a.native_datafusion/extended.txt index 9091536e96..cba125f412 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q35a.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q35a.native_datafusion/extended.txt @@ -1,65 +1,66 @@ -TakeOrderedAndProject -+- HashAggregate - +- CometNativeColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- BroadcastHashJoin - : : : :- CometNativeColumnarToRow - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.customer - : : : +- BroadcastExchange - : : : +- Project - : : : +- BroadcastHashJoin - : : : :- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- Union - : : :- Project - : : : +- BroadcastHashJoin - : : : :- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- ReusedSubquery - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- Project - : : +- BroadcastHashJoin - : : :- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : +- ReusedSubquery - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- CometNativeColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.customer_address - +- BroadcastExchange - +- CometNativeColumnarToRow - +- CometProject - +- CometFilter - +- CometNativeScan parquet spark_catalog.default.customer_demographics +CometNativeColumnarToRow ++- CometTakeOrderedAndProject + +- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.customer + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometNativeScan parquet spark_catalog.default.store_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometNativeColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometUnion + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometNativeScan parquet spark_catalog.default.web_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometNativeColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometNativeColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.customer_address + +- CometBroadcastExchange + +- CometProject + +- CometFilter + +- CometNativeScan parquet spark_catalog.default.customer_demographics -Comet accelerated 21 out of 52 eligible operators (40%). Final plan contains 11 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 54 out of 60 eligible operators (90%). Final plan contains 4 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q36a.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q36a.native_datafusion/extended.txt index 1fc86dc5a8..0a69bca968 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q36a.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q36a.native_datafusion/extended.txt @@ -5,118 +5,101 @@ TakeOrderedAndProject +- CometSort +- CometExchange +- CometHashAggregate - +- CometColumnarExchange - +- HashAggregate - +- Union - :- HashAggregate - : +- CometNativeColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- Filter - : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.item - : +- BroadcastExchange - : +- CometNativeColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.store - :- HashAggregate - : +- CometNativeColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- HashAggregate - : +- CometNativeColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- Filter - : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.item - : +- BroadcastExchange - : +- CometNativeColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.store - +- HashAggregate - +- CometNativeColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- HashAggregate - +- CometNativeColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Filter - : : : +- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- CometNativeColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.item - +- BroadcastExchange - +- CometNativeColumnarToRow - +- CometProject - +- CometFilter - +- CometNativeScan parquet spark_catalog.default.store + +- CometExchange + +- CometHashAggregate + +- CometUnion + :- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometNativeColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.item + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.store + :- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometNativeColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.item + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.store + +- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometNativeColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.item + +- CometBroadcastExchange + +- CometProject + +- CometFilter + +- CometNativeScan parquet spark_catalog.default.store -Comet accelerated 45 out of 99 eligible operators (45%). Final plan contains 21 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 90 out of 99 eligible operators (90%). Final plan contains 4 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q47.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q47.native_datafusion/extended.txt index ab44bbef5c..0886059452 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q47.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q47.native_datafusion/extended.txt @@ -10,111 +10,96 @@ TakeOrderedAndProject : : +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] : : +- CometNativeColumnarToRow : : +- CometSort - : : +- CometColumnarExchange - : : +- HashAggregate - : : +- CometNativeColumnarToRow - : : +- CometColumnarExchange - : : +- HashAggregate - : : +- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- Project - : : : : +- BroadcastHashJoin - : : : : :- CometNativeColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : +- BroadcastExchange - : : : : +- Filter - : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.store + : : +- CometExchange + : : +- CometHashAggregate + : : +- CometExchange + : : +- CometHashAggregate + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : : +- CometBroadcastExchange + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometNativeColumnarToRow + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.store : +- BroadcastExchange : +- Project : +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] : +- CometNativeColumnarToRow : +- CometSort - : +- CometColumnarExchange - : +- HashAggregate - : +- CometNativeColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : +- BroadcastExchange - : : : +- Filter - : : : +- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- CometNativeColumnarToRow - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.store + : +- CometExchange + : +- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : +- CometBroadcastExchange + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometNativeColumnarToRow + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.store +- BroadcastExchange +- Project +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] +- CometNativeColumnarToRow +- CometSort - +- CometColumnarExchange - +- HashAggregate - +- CometNativeColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.item - : : +- BroadcastExchange - : : +- Filter - : : +- ColumnarToRow - : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- CometNativeColumnarToRow - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.date_dim - +- BroadcastExchange - +- CometNativeColumnarToRow - +- CometFilter - +- CometNativeScan parquet spark_catalog.default.store + +- CometExchange + +- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.item + : : +- CometBroadcastExchange + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : +- SubqueryBroadcast + : : +- BroadcastExchange + : : +- CometNativeColumnarToRow + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.date_dim + +- CometBroadcastExchange + +- CometFilter + +- CometNativeScan parquet spark_catalog.default.store -Comet accelerated 36 out of 97 eligible operators (37%). Final plan contains 21 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 75 out of 97 eligible operators (77%). Final plan contains 6 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q49.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q49.native_datafusion/extended.txt index 0090739d55..6d7dadcde9 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q49.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q49.native_datafusion/extended.txt @@ -11,35 +11,31 @@ CometNativeColumnarToRow : +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] : +- CometNativeColumnarToRow : +- CometSort - : +- CometColumnarExchange - : +- HashAggregate - : +- CometNativeColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- BroadcastExchange - : : : +- Project - : : : +- Filter - : : : +- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- CometNativeColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.web_returns - : +- BroadcastExchange - : +- CometNativeColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometExchange + : +- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.web_sales + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometNativeColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.web_returns + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.date_dim :- Project : +- Filter : +- Window @@ -47,30 +43,31 @@ CometNativeColumnarToRow : +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] : +- CometNativeColumnarToRow : +- CometSort - : +- CometColumnarExchange - : +- HashAggregate - : +- CometNativeColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- BroadcastExchange - : : : +- Project - : : : +- Filter - : : : +- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : +- ReusedSubquery - : : +- CometNativeColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.catalog_returns - : +- BroadcastExchange - : +- CometNativeColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometExchange + : +- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometNativeColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.catalog_returns + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.date_dim +- Project +- Filter +- Window @@ -78,29 +75,30 @@ CometNativeColumnarToRow +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] +- CometNativeColumnarToRow +- CometSort - +- CometColumnarExchange - +- HashAggregate - +- CometNativeColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- BroadcastExchange - : : +- Project - : : +- Filter - : : +- ColumnarToRow - : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : +- ReusedSubquery - : +- CometNativeColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.store_returns - +- BroadcastExchange - +- CometNativeColumnarToRow - +- CometProject - +- CometFilter - +- CometNativeScan parquet spark_catalog.default.date_dim + +- CometExchange + +- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : +- SubqueryBroadcast + : : +- BroadcastExchange + : : +- CometNativeColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.store_returns + +- CometBroadcastExchange + +- CometProject + +- CometFilter + +- CometNativeScan parquet spark_catalog.default.date_dim -Comet accelerated 33 out of 87 eligible operators (37%). Final plan contains 17 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 72 out of 95 eligible operators (75%). Final plan contains 7 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q51a.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q51a.native_datafusion/extended.txt index 92ee33eda6..d9b9186c1e 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q51a.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q51a.native_datafusion/extended.txt @@ -23,53 +23,47 @@ TakeOrderedAndProject : : : +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] : : : +- CometNativeColumnarToRow : : : +- CometSort - : : : +- CometColumnarExchange - : : : +- HashAggregate - : : : +- CometNativeColumnarToRow - : : : +- CometColumnarExchange - : : : +- HashAggregate - : : : +- Project - : : : +- BroadcastHashJoin - : : : :- Filter - : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometExchange + : : : +- CometHashAggregate + : : : +- CometExchange + : : : +- CometHashAggregate + : : : +- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.web_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometNativeColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- BroadcastExchange : : +- Project : : +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] : : +- CometNativeColumnarToRow : : +- CometSort - : : +- CometColumnarExchange - : : +- HashAggregate - : : +- CometNativeColumnarToRow - : : +- CometColumnarExchange - : : +- HashAggregate - : : +- Project - : : +- BroadcastHashJoin - : : :- Filter - : : : +- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometExchange + : : +- CometHashAggregate + : : +- CometExchange + : : +- CometHashAggregate + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.web_sales + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometNativeColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim : +- CometSort : +- CometColumnarExchange : +- HashAggregate @@ -82,43 +76,47 @@ TakeOrderedAndProject : : +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] : : +- CometNativeColumnarToRow : : +- CometSort - : : +- CometColumnarExchange - : : +- HashAggregate - : : +- CometNativeColumnarToRow - : : +- CometColumnarExchange - : : +- HashAggregate - : : +- Project - : : +- BroadcastHashJoin - : : :- Filter - : : : +- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : +- ReusedSubquery - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometExchange + : : +- CometHashAggregate + : : +- CometExchange + : : +- CometHashAggregate + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometNativeColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim : +- BroadcastExchange : +- Project : +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] : +- CometNativeColumnarToRow : +- CometSort - : +- CometColumnarExchange - : +- HashAggregate - : +- CometNativeColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- Filter - : : +- ColumnarToRow - : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : +- ReusedSubquery - : +- BroadcastExchange - : +- CometNativeColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometExchange + : +- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : +- SubqueryBroadcast + : : +- BroadcastExchange + : : +- CometNativeColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.date_dim +- BroadcastExchange +- Project +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] @@ -140,53 +138,47 @@ TakeOrderedAndProject : : +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] : : +- CometNativeColumnarToRow : : +- CometSort - : : +- CometColumnarExchange - : : +- HashAggregate - : : +- CometNativeColumnarToRow - : : +- CometColumnarExchange - : : +- HashAggregate - : : +- Project - : : +- BroadcastHashJoin - : : :- Filter - : : : +- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometExchange + : : +- CometHashAggregate + : : +- CometExchange + : : +- CometHashAggregate + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.web_sales + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometNativeColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim : +- BroadcastExchange : +- Project : +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] : +- CometNativeColumnarToRow : +- CometSort - : +- CometColumnarExchange - : +- HashAggregate - : +- CometNativeColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- Filter - : : +- ColumnarToRow - : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- CometNativeColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometExchange + : +- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.web_sales + : : +- SubqueryBroadcast + : : +- BroadcastExchange + : : +- CometNativeColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.date_dim +- CometSort +- CometColumnarExchange +- HashAggregate @@ -199,42 +191,46 @@ TakeOrderedAndProject : +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] : +- CometNativeColumnarToRow : +- CometSort - : +- CometColumnarExchange - : +- HashAggregate - : +- CometNativeColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- Filter - : : +- ColumnarToRow - : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : +- ReusedSubquery - : +- BroadcastExchange - : +- CometNativeColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometExchange + : +- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : +- SubqueryBroadcast + : : +- BroadcastExchange + : : +- CometNativeColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.date_dim +- BroadcastExchange +- Project +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] +- CometNativeColumnarToRow +- CometSort - +- CometColumnarExchange - +- HashAggregate - +- CometNativeColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Project - +- BroadcastHashJoin - :- Filter - : +- ColumnarToRow - : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : +- ReusedSubquery - +- BroadcastExchange - +- CometNativeColumnarToRow - +- CometProject - +- CometFilter - +- CometNativeScan parquet spark_catalog.default.date_dim + +- CometExchange + +- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometProject + +- CometBroadcastHashJoin + :- CometFilter + : +- CometNativeScan parquet spark_catalog.default.store_sales + : +- SubqueryBroadcast + : +- BroadcastExchange + : +- CometNativeColumnarToRow + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.date_dim + +- CometBroadcastExchange + +- CometProject + +- CometFilter + +- CometNativeScan parquet spark_catalog.default.date_dim -Comet accelerated 82 out of 196 eligible operators (41%). Final plan contains 42 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 150 out of 212 eligible operators (70%). Final plan contains 22 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q57.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q57.native_datafusion/extended.txt index 66c510aaa0..024bfeeeea 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q57.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q57.native_datafusion/extended.txt @@ -10,111 +10,96 @@ TakeOrderedAndProject : : +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] : : +- CometNativeColumnarToRow : : +- CometSort - : : +- CometColumnarExchange - : : +- HashAggregate - : : +- CometNativeColumnarToRow - : : +- CometColumnarExchange - : : +- HashAggregate - : : +- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- Project - : : : : +- BroadcastHashJoin - : : : : :- CometNativeColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : +- BroadcastExchange - : : : : +- Filter - : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.call_center + : : +- CometExchange + : : +- CometHashAggregate + : : +- CometExchange + : : +- CometHashAggregate + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : : +- CometBroadcastExchange + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometNativeColumnarToRow + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.call_center : +- BroadcastExchange : +- Project : +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] : +- CometNativeColumnarToRow : +- CometSort - : +- CometColumnarExchange - : +- HashAggregate - : +- CometNativeColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : +- BroadcastExchange - : : : +- Filter - : : : +- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- CometNativeColumnarToRow - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.call_center + : +- CometExchange + : +- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : +- CometBroadcastExchange + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometNativeColumnarToRow + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.call_center +- BroadcastExchange +- Project +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] +- CometNativeColumnarToRow +- CometSort - +- CometColumnarExchange - +- HashAggregate - +- CometNativeColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.item - : : +- BroadcastExchange - : : +- Filter - : : +- ColumnarToRow - : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- CometNativeColumnarToRow - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.date_dim - +- BroadcastExchange - +- CometNativeColumnarToRow - +- CometFilter - +- CometNativeScan parquet spark_catalog.default.call_center + +- CometExchange + +- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.item + : : +- CometBroadcastExchange + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : +- SubqueryBroadcast + : : +- BroadcastExchange + : : +- CometNativeColumnarToRow + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.date_dim + +- CometBroadcastExchange + +- CometFilter + +- CometNativeScan parquet spark_catalog.default.call_center -Comet accelerated 36 out of 97 eligible operators (37%). Final plan contains 21 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 75 out of 97 eligible operators (77%). Final plan contains 6 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q5a.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q5a.native_datafusion/extended.txt index ec2eef025a..9cc06fb361 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q5a.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q5a.native_datafusion/extended.txt @@ -1,322 +1,344 @@ CometNativeColumnarToRow +- CometTakeOrderedAndProject +- CometHashAggregate - +- CometColumnarExchange - +- HashAggregate - +- Union - :- HashAggregate - : +- CometNativeColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Union - : :- HashAggregate - : : +- CometNativeColumnarToRow - : : +- CometColumnarExchange - : : +- HashAggregate - : : +- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- Union - : : : : :- Project - : : : : : +- Filter - : : : : : +- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : +- Project - : : : : +- Filter - : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.store_returns [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- ReusedSubquery - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.store - : :- HashAggregate - : : +- CometNativeColumnarToRow - : : +- CometColumnarExchange - : : +- HashAggregate - : : +- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- Union - : : : : :- Project - : : : : : +- Filter - : : : : : +- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : +- ReusedSubquery - : : : : +- Project - : : : : +- Filter - : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.catalog_returns [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- ReusedSubquery - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.catalog_page - : +- HashAggregate - : +- CometNativeColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Union - : : : :- Project - : : : : +- Filter - : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- ReusedSubquery - : : : +- Project - : : : +- BroadcastHashJoin - : : : :- BroadcastExchange - : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.web_returns [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- ReusedSubquery - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- CometNativeColumnarToRow + +- CometExchange + +- CometHashAggregate + +- CometUnion + :- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometUnion + : :- CometHashAggregate + : : +- CometExchange + : : +- CometHashAggregate + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometUnion + : : : : :- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometNativeColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.store_returns + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometNativeColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.store + : :- CometHashAggregate + : : +- CometExchange + : : +- CometHashAggregate + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometUnion + : : : : :- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometNativeColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.catalog_returns + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometNativeColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.catalog_page + : +- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometUnion + : : : :- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.web_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometNativeColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometBroadcastExchange + : : : : +- CometNativeScan parquet spark_catalog.default.web_returns + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometNativeColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.web_sales + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.web_site + :- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometUnion + : :- CometHashAggregate + : : +- CometExchange + : : +- CometHashAggregate + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometUnion + : : : : :- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometNativeColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.store_returns + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometNativeColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.store + : :- CometHashAggregate + : : +- CometExchange + : : +- CometHashAggregate + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometUnion + : : : : :- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometNativeColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.catalog_returns + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometNativeColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.catalog_page + : +- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometUnion + : : : :- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.web_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometNativeColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometBroadcastExchange + : : : : +- CometNativeScan parquet spark_catalog.default.web_returns + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometNativeColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.web_sales + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange : +- CometProject : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.web_site - :- HashAggregate - : +- CometNativeColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- HashAggregate - : +- CometNativeColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Union - : :- HashAggregate - : : +- CometNativeColumnarToRow - : : +- CometColumnarExchange - : : +- HashAggregate - : : +- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- Union - : : : : :- Project - : : : : : +- Filter - : : : : : +- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : +- Project - : : : : +- Filter - : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.store_returns [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- ReusedSubquery - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.store - : :- HashAggregate - : : +- CometNativeColumnarToRow - : : +- CometColumnarExchange - : : +- HashAggregate - : : +- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- Union - : : : : :- Project - : : : : : +- Filter - : : : : : +- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : +- ReusedSubquery - : : : : +- Project - : : : : +- Filter - : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.catalog_returns [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- ReusedSubquery - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.catalog_page - : +- HashAggregate - : +- CometNativeColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Union - : : : :- Project - : : : : +- Filter - : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- ReusedSubquery - : : : +- Project - : : : +- BroadcastHashJoin - : : : :- BroadcastExchange - : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.web_returns [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- ReusedSubquery - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- CometNativeColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.web_site - +- HashAggregate - +- CometNativeColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- HashAggregate - +- CometNativeColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Union - :- HashAggregate - : +- CometNativeColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Union - : : : :- Project - : : : : +- Filter - : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- Project - : : : +- Filter - : : : +- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.store_returns [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : +- ReusedSubquery - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- CometNativeColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.store - :- HashAggregate - : +- CometNativeColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Union - : : : :- Project - : : : : +- Filter - : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- ReusedSubquery - : : : +- Project - : : : +- Filter - : : : +- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.catalog_returns [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : +- ReusedSubquery - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- CometNativeColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.catalog_page - +- HashAggregate - +- CometNativeColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- Union - : : :- Project - : : : +- Filter - : : : +- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : +- ReusedSubquery - : : +- Project - : : +- BroadcastHashJoin - : : :- BroadcastExchange - : : : +- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.web_returns [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : +- ReusedSubquery - : : +- CometNativeColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.web_sales - : +- BroadcastExchange - : +- CometNativeColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.date_dim - +- BroadcastExchange - +- CometNativeColumnarToRow - +- CometProject - +- CometFilter - +- CometNativeScan parquet spark_catalog.default.web_site + +- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometUnion + :- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometUnion + : : : :- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometNativeColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.store_returns + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometNativeColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.store + :- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometUnion + : : : :- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometNativeColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.catalog_returns + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometNativeColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.catalog_page + +- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometUnion + : : :- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.web_sales + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometNativeColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometBroadcastExchange + : : : +- CometNativeScan parquet spark_catalog.default.web_returns + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometNativeColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.web_sales + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.date_dim + +- CometBroadcastExchange + +- CometProject + +- CometFilter + +- CometNativeScan parquet spark_catalog.default.web_site -Comet accelerated 89 out of 263 eligible operators (33%). Final plan contains 57 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 287 out of 323 eligible operators (88%). Final plan contains 19 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q6.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q6.native_datafusion/extended.txt index 44f11f06fa..d5de400618 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q6.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q6.native_datafusion/extended.txt @@ -1,70 +1,73 @@ -TakeOrderedAndProject -+- Filter - +- HashAggregate - +- CometNativeColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.customer_address - : : : +- CometBroadcastExchange - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.customer - : : +- BroadcastExchange - : : +- Filter - : : +- ColumnarToRow - : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometProject - : : +- CometFilter - : : : +- ReusedSubquery - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- Subquery - : : +- CometNativeColumnarToRow - : : +- CometHashAggregate - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- CometNativeColumnarToRow - : +- CometProject - : +- CometFilter - : : +- ReusedSubquery - : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- Subquery - : +- CometNativeColumnarToRow - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.date_dim - +- BroadcastExchange - +- CometNativeColumnarToRow - +- CometProject - +- CometBroadcastHashJoin - :- CometFilter - : +- CometNativeScan parquet spark_catalog.default.item - +- CometBroadcastExchange - +- CometFilter - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometFilter - +- CometNativeScan parquet spark_catalog.default.item +CometNativeColumnarToRow ++- CometTakeOrderedAndProject + +- CometFilter + +- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.customer_address + : : : +- CometBroadcastExchange + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.customer + : : +- CometBroadcastExchange + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : +- SubqueryBroadcast + : : +- BroadcastExchange + : : +- CometNativeColumnarToRow + : : +- CometProject + : : +- CometFilter + : : : +- Subquery + : : : +- CometNativeColumnarToRow + : : : +- CometHashAggregate + : : : +- CometExchange + : : : +- CometHashAggregate + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- Subquery + : : +- CometNativeColumnarToRow + : : +- CometHashAggregate + : : +- CometExchange + : : +- CometHashAggregate + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : : +- ReusedSubquery + : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- Subquery + : +- CometNativeColumnarToRow + : +- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.date_dim + +- CometBroadcastExchange + +- CometProject + +- CometBroadcastHashJoin + :- CometFilter + : +- CometNativeScan parquet spark_catalog.default.item + +- CometBroadcastExchange + +- CometFilter + +- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometProject + +- CometFilter + +- CometNativeScan parquet spark_catalog.default.item -Comet accelerated 39 out of 60 eligible operators (65%). Final plan contains 8 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 60 out of 66 eligible operators (90%). Final plan contains 5 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q64.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q64.native_datafusion/extended.txt index 98db64e5b0..f1fe6fb3e3 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q64.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q64.native_datafusion/extended.txt @@ -40,22 +40,20 @@ CometNativeColumnarToRow : : : : : : : : : : : : : : : :- CometProject : : : : : : : : : : : : : : : : +- CometSortMergeJoin : : : : : : : : : : : : : : : : :- CometSort - : : : : : : : : : : : : : : : : : +- CometColumnarExchange - : : : : : : : : : : : : : : : : : +- Project - : : : : : : : : : : : : : : : : : +- BroadcastHashJoin - : : : : : : : : : : : : : : : : : :- BroadcastExchange - : : : : : : : : : : : : : : : : : : +- Filter - : : : : : : : : : : : : : : : : : : +- ColumnarToRow - : : : : : : : : : : : : : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : : : : : : : : : : : : : : +- SubqueryBroadcast - : : : : : : : : : : : : : : : : : : +- BroadcastExchange - : : : : : : : : : : : : : : : : : : +- CometNativeColumnarToRow - : : : : : : : : : : : : : : : : : : +- CometFilter - : : : : : : : : : : : : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : : : : : : : : : : : : : : +- CometNativeColumnarToRow - : : : : : : : : : : : : : : : : : +- CometProject - : : : : : : : : : : : : : : : : : +- CometFilter - : : : : : : : : : : : : : : : : : +- CometNativeScan parquet spark_catalog.default.store_returns + : : : : : : : : : : : : : : : : : +- CometExchange + : : : : : : : : : : : : : : : : : +- CometProject + : : : : : : : : : : : : : : : : : +- CometBroadcastHashJoin + : : : : : : : : : : : : : : : : : :- CometBroadcastExchange + : : : : : : : : : : : : : : : : : : +- CometFilter + : : : : : : : : : : : : : : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : : : : : : : : : : : : : : : +- SubqueryBroadcast + : : : : : : : : : : : : : : : : : : +- BroadcastExchange + : : : : : : : : : : : : : : : : : : +- CometNativeColumnarToRow + : : : : : : : : : : : : : : : : : : +- CometFilter + : : : : : : : : : : : : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : : : : : : : : : : : : +- CometProject + : : : : : : : : : : : : : : : : : +- CometFilter + : : : : : : : : : : : : : : : : : +- CometNativeScan parquet spark_catalog.default.store_returns : : : : : : : : : : : : : : : : +- CometSort : : : : : : : : : : : : : : : : +- CometProject : : : : : : : : : : : : : : : : +- CometFilter @@ -162,22 +160,20 @@ CometNativeColumnarToRow : : : : : : : : : : : : : : :- CometProject : : : : : : : : : : : : : : : +- CometSortMergeJoin : : : : : : : : : : : : : : : :- CometSort - : : : : : : : : : : : : : : : : +- CometColumnarExchange - : : : : : : : : : : : : : : : : +- Project - : : : : : : : : : : : : : : : : +- BroadcastHashJoin - : : : : : : : : : : : : : : : : :- BroadcastExchange - : : : : : : : : : : : : : : : : : +- Filter - : : : : : : : : : : : : : : : : : +- ColumnarToRow - : : : : : : : : : : : : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : : : : : : : : : : : : : +- SubqueryBroadcast - : : : : : : : : : : : : : : : : : +- BroadcastExchange - : : : : : : : : : : : : : : : : : +- CometNativeColumnarToRow - : : : : : : : : : : : : : : : : : +- CometFilter - : : : : : : : : : : : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : : : : : : : : : : : : : +- CometNativeColumnarToRow - : : : : : : : : : : : : : : : : +- CometProject - : : : : : : : : : : : : : : : : +- CometFilter - : : : : : : : : : : : : : : : : +- CometNativeScan parquet spark_catalog.default.store_returns + : : : : : : : : : : : : : : : : +- CometExchange + : : : : : : : : : : : : : : : : +- CometProject + : : : : : : : : : : : : : : : : +- CometBroadcastHashJoin + : : : : : : : : : : : : : : : : :- CometBroadcastExchange + : : : : : : : : : : : : : : : : : +- CometFilter + : : : : : : : : : : : : : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : : : : : : : : : : : : : : +- SubqueryBroadcast + : : : : : : : : : : : : : : : : : +- BroadcastExchange + : : : : : : : : : : : : : : : : : +- CometNativeColumnarToRow + : : : : : : : : : : : : : : : : : +- CometFilter + : : : : : : : : : : : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : : : : : : : : : : : +- CometProject + : : : : : : : : : : : : : : : : +- CometFilter + : : : : : : : : : : : : : : : : +- CometNativeScan parquet spark_catalog.default.store_returns : : : : : : : : : : : : : : : +- CometSort : : : : : : : : : : : : : : : +- CometProject : : : : : : : : : : : : : : : +- CometFilter @@ -248,4 +244,4 @@ CometNativeColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.item -Comet accelerated 228 out of 242 eligible operators (94%). Final plan contains 7 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 238 out of 242 eligible operators (98%). Final plan contains 3 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q67a.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q67a.native_datafusion/extended.txt index bf553c1971..abf8dad401 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q67a.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q67a.native_datafusion/extended.txt @@ -5,346 +5,294 @@ TakeOrderedAndProject +- CometNativeColumnarToRow +- CometSort +- CometColumnarExchange - +- WindowGroupLimit - +- Sort - +- Union - :- HashAggregate - : +- CometNativeColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- Filter - : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.store - : +- BroadcastExchange - : +- CometNativeColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.item - :- HashAggregate - : +- CometNativeColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- HashAggregate - : +- CometNativeColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- Filter - : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.store - : +- BroadcastExchange - : +- CometNativeColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.item - :- HashAggregate - : +- CometNativeColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- HashAggregate - : +- CometNativeColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- Filter - : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.store - : +- BroadcastExchange - : +- CometNativeColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.item - :- HashAggregate - : +- CometNativeColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- HashAggregate - : +- CometNativeColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- Filter - : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.store - : +- BroadcastExchange - : +- CometNativeColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.item - :- HashAggregate - : +- CometNativeColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- HashAggregate - : +- CometNativeColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- Filter - : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.store - : +- BroadcastExchange - : +- CometNativeColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.item - :- HashAggregate - : +- CometNativeColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- HashAggregate - : +- CometNativeColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- Filter - : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.store - : +- BroadcastExchange - : +- CometNativeColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.item - :- HashAggregate - : +- CometNativeColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- HashAggregate - : +- CometNativeColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- Filter - : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.store - : +- BroadcastExchange - : +- CometNativeColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.item - :- HashAggregate - : +- CometNativeColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- HashAggregate - : +- CometNativeColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- Filter - : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.store - : +- BroadcastExchange - : +- CometNativeColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.item - +- HashAggregate - +- CometNativeColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- HashAggregate - +- CometNativeColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Filter - : : : +- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- CometNativeColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.store - +- BroadcastExchange - +- CometNativeColumnarToRow - +- CometProject - +- CometFilter - +- CometNativeScan parquet spark_catalog.default.item + +- WindowGroupLimit [COMET: WindowGroupLimit is not supported] + +- CometNativeColumnarToRow + +- CometSort + +- CometUnion + :- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometNativeColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.store + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.item + :- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometNativeColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.store + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.item + :- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometNativeColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.store + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.item + :- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometNativeColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.store + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.item + :- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometNativeColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.store + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.item + :- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometNativeColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.store + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.item + :- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometNativeColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.store + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.item + :- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometNativeColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.store + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.item + +- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometNativeColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.store + +- CometBroadcastExchange + +- CometProject + +- CometFilter + +- CometNativeScan parquet spark_catalog.default.item -Comet accelerated 127 out of 285 eligible operators (44%). Final plan contains 63 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 262 out of 285 eligible operators (91%). Final plan contains 11 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q70a.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q70a.native_datafusion/extended.txt index 3214d035b2..4ba9e6eb00 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q70a.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q70a.native_datafusion/extended.txt @@ -14,19 +14,18 @@ TakeOrderedAndProject : +- HashAggregate : +- Project : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Filter - : : : +- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow + : :- CometNativeColumnarToRow + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometNativeColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange : : +- CometProject : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -40,30 +39,32 @@ TakeOrderedAndProject : +- Project : +- Filter : +- Window - : +- WindowGroupLimit - : +- Sort - : +- HashAggregate - : +- CometNativeColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Filter - : : : +- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : +- ReusedSubquery - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.store - : +- BroadcastExchange - : +- CometNativeColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- WindowGroupLimit [COMET: WindowGroupLimit is not supported] + : +- CometNativeColumnarToRow + : +- CometSort + : +- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometNativeColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.store + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.date_dim :- HashAggregate : +- CometNativeColumnarToRow : +- CometColumnarExchange @@ -74,19 +75,18 @@ TakeOrderedAndProject : +- HashAggregate : +- Project : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Filter - : : : +- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow + : :- CometNativeColumnarToRow + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometNativeColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange : : +- CometProject : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -100,30 +100,32 @@ TakeOrderedAndProject : +- Project : +- Filter : +- Window - : +- WindowGroupLimit - : +- Sort - : +- HashAggregate - : +- CometNativeColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Filter - : : : +- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : +- ReusedSubquery - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.store - : +- BroadcastExchange - : +- CometNativeColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- WindowGroupLimit [COMET: WindowGroupLimit is not supported] + : +- CometNativeColumnarToRow + : +- CometSort + : +- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometNativeColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.store + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.date_dim +- HashAggregate +- CometNativeColumnarToRow +- CometColumnarExchange @@ -134,19 +136,18 @@ TakeOrderedAndProject +- HashAggregate +- Project +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- Filter - : : +- ColumnarToRow - : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- CometNativeColumnarToRow + :- CometNativeColumnarToRow + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : +- SubqueryBroadcast + : : +- BroadcastExchange + : : +- CometNativeColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange : +- CometProject : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -160,29 +161,31 @@ TakeOrderedAndProject +- Project +- Filter +- Window - +- WindowGroupLimit - +- Sort - +- HashAggregate - +- CometNativeColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- Filter - : : +- ColumnarToRow - : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : +- ReusedSubquery - : +- BroadcastExchange - : +- CometNativeColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.store - +- BroadcastExchange - +- CometNativeColumnarToRow - +- CometProject - +- CometFilter - +- CometNativeScan parquet spark_catalog.default.date_dim + +- WindowGroupLimit [COMET: WindowGroupLimit is not supported] + +- CometNativeColumnarToRow + +- CometSort + +- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : +- SubqueryBroadcast + : : +- BroadcastExchange + : : +- CometNativeColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.store + +- CometBroadcastExchange + +- CometProject + +- CometFilter + +- CometNativeScan parquet spark_catalog.default.date_dim -Comet accelerated 54 out of 156 eligible operators (34%). Final plan contains 30 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 111 out of 168 eligible operators (66%). Final plan contains 21 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q72.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q72.native_datafusion/extended.txt index 62c2b17b7e..623a68ddab 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q72.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q72.native_datafusion/extended.txt @@ -6,77 +6,67 @@ CometNativeColumnarToRow +- CometProject +- CometSortMergeJoin :- CometSort - : +- CometColumnarExchange - : +- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- Project - : : : : +- BroadcastHashJoin - : : : : :- Project - : : : : : +- BroadcastHashJoin - : : : : : :- Project - : : : : : : +- BroadcastHashJoin - : : : : : : :- Project - : : : : : : : +- BroadcastHashJoin - : : : : : : : :- Project - : : : : : : : : +- BroadcastHashJoin - : : : : : : : : :- Project - : : : : : : : : : +- BroadcastHashJoin - : : : : : : : : : :- Filter - : : : : : : : : : : +- ColumnarToRow - : : : : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : : : : : : +- SubqueryBroadcast - : : : : : : : : : : +- BroadcastExchange - : : : : : : : : : : +- CometNativeColumnarToRow - : : : : : : : : : : +- CometProject - : : : : : : : : : : +- CometFilter - : : : : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : : : : : : +- BroadcastExchange - : : : : : : : : : +- CometNativeColumnarToRow - : : : : : : : : : +- CometFilter - : : : : : : : : : +- CometNativeScan parquet spark_catalog.default.inventory - : : : : : : : : +- BroadcastExchange - : : : : : : : : +- CometNativeColumnarToRow - : : : : : : : : +- CometFilter - : : : : : : : : +- CometNativeScan parquet spark_catalog.default.warehouse - : : : : : : : +- BroadcastExchange - : : : : : : : +- CometNativeColumnarToRow - : : : : : : : +- CometFilter - : : : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : : : +- BroadcastExchange - : : : : : : +- CometNativeColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.customer_demographics - : : : : : +- BroadcastExchange - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.household_demographics - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- CometNativeColumnarToRow - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.promotion + : +- CometExchange + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometProject + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometProject + : : : : : : +- CometBroadcastHashJoin + : : : : : : :- CometProject + : : : : : : : +- CometBroadcastHashJoin + : : : : : : : :- CometProject + : : : : : : : : +- CometBroadcastHashJoin + : : : : : : : : :- CometProject + : : : : : : : : : +- CometBroadcastHashJoin + : : : : : : : : : :- CometFilter + : : : : : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : : : : : : : +- SubqueryBroadcast + : : : : : : : : : : +- BroadcastExchange + : : : : : : : : : : +- CometNativeColumnarToRow + : : : : : : : : : : +- CometProject + : : : : : : : : : : +- CometFilter + : : : : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : : : : +- CometBroadcastExchange + : : : : : : : : : +- CometFilter + : : : : : : : : : +- CometNativeScan parquet spark_catalog.default.inventory + : : : : : : : : +- CometBroadcastExchange + : : : : : : : : +- CometFilter + : : : : : : : : +- CometNativeScan parquet spark_catalog.default.warehouse + : : : : : : : +- CometBroadcastExchange + : : : : : : : +- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : : : : +- CometBroadcastExchange + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.customer_demographics + : : : : : +- CometBroadcastExchange + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.household_demographics + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.promotion +- CometSort +- CometExchange +- CometProject +- CometFilter +- CometNativeScan parquet spark_catalog.default.catalog_returns -Comet accelerated 37 out of 68 eligible operators (54%). Final plan contains 12 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 66 out of 68 eligible operators (97%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q74.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q74.native_datafusion/extended.txt index 5d218998be..51a3f5ca94 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q74.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q74.native_datafusion/extended.txt @@ -1,105 +1,98 @@ -TakeOrderedAndProject -+- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- BroadcastHashJoin - : : :- Filter - : : : +- HashAggregate - : : : +- CometNativeColumnarToRow - : : : +- CometColumnarExchange - : : : +- HashAggregate - : : : +- Project - : : : +- BroadcastHashJoin - : : : :- Project - : : : : +- BroadcastHashJoin - : : : : :- CometNativeColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.customer - : : : : +- BroadcastExchange - : : : : +- Filter - : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- HashAggregate - : : +- CometNativeColumnarToRow - : : +- CometColumnarExchange - : : +- HashAggregate - : : +- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.customer - : : : +- BroadcastExchange - : : : +- Filter - : : : +- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- Filter - : +- HashAggregate - : +- CometNativeColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.customer - : : +- BroadcastExchange - : : +- Filter - : : +- ColumnarToRow - : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : +- ReusedSubquery - : +- BroadcastExchange - : +- CometNativeColumnarToRow - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.date_dim - +- BroadcastExchange - +- HashAggregate - +- CometNativeColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- CometNativeColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.customer - : +- BroadcastExchange - : +- Filter - : +- ColumnarToRow - : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : +- ReusedSubquery - +- BroadcastExchange - +- CometNativeColumnarToRow - +- CometFilter - +- CometNativeScan parquet spark_catalog.default.date_dim +CometNativeColumnarToRow ++- CometTakeOrderedAndProject + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometHashAggregate + : : : +- CometExchange + : : : +- CometHashAggregate + : : : +- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.customer + : : : : +- CometBroadcastExchange + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometNativeColumnarToRow + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometHashAggregate + : : +- CometExchange + : : +- CometHashAggregate + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.customer + : : : +- CometBroadcastExchange + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometNativeColumnarToRow + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometFilter + : +- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.customer + : : +- CometBroadcastExchange + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.web_sales + : : +- SubqueryBroadcast + : : +- BroadcastExchange + : : +- CometNativeColumnarToRow + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.date_dim + +- CometBroadcastExchange + +- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.customer + : +- CometBroadcastExchange + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.web_sales + : +- SubqueryBroadcast + : +- BroadcastExchange + : +- CometNativeColumnarToRow + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.date_dim + +- CometBroadcastExchange + +- CometFilter + +- CometNativeScan parquet spark_catalog.default.date_dim -Comet accelerated 28 out of 85 eligible operators (32%). Final plan contains 18 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 83 out of 91 eligible operators (91%). Final plan contains 5 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q75.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q75.native_datafusion/extended.txt index 42106a0ffd..a943308f1e 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q75.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q75.native_datafusion/extended.txt @@ -15,28 +15,25 @@ CometNativeColumnarToRow : :- CometProject : : +- CometSortMergeJoin : : :- CometSort - : : : +- CometColumnarExchange - : : : +- Project - : : : +- BroadcastHashJoin - : : : :- Project - : : : : +- BroadcastHashJoin - : : : : :- Filter - : : : : : +- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometExchange + : : : +- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometNativeColumnarToRow + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : +- CometBroadcastExchange + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- CometSort : : +- CometExchange : : +- CometProject @@ -45,24 +42,25 @@ CometNativeColumnarToRow : :- CometProject : : +- CometSortMergeJoin : : :- CometSort - : : : +- CometColumnarExchange - : : : +- Project - : : : +- BroadcastHashJoin - : : : :- Project - : : : : +- BroadcastHashJoin - : : : : :- Filter - : : : : : +- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : +- ReusedSubquery - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometExchange + : : : +- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometNativeColumnarToRow + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : +- CometBroadcastExchange + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- CometSort : : +- CometExchange : : +- CometProject @@ -71,24 +69,25 @@ CometNativeColumnarToRow : +- CometProject : +- CometSortMergeJoin : :- CometSort - : : +- CometColumnarExchange - : : +- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- Filter - : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- ReusedSubquery - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.item - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometExchange + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.web_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometNativeColumnarToRow + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.item + : : +- CometBroadcastExchange + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim : +- CometSort : +- CometExchange : +- CometProject @@ -107,28 +106,25 @@ CometNativeColumnarToRow :- CometProject : +- CometSortMergeJoin : :- CometSort - : : +- CometColumnarExchange - : : +- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- Filter - : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.item - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometExchange + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometNativeColumnarToRow + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.item + : : +- CometBroadcastExchange + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim : +- CometSort : +- CometExchange : +- CometProject @@ -137,24 +133,25 @@ CometNativeColumnarToRow :- CometProject : +- CometSortMergeJoin : :- CometSort - : : +- CometColumnarExchange - : : +- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- Filter - : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- ReusedSubquery - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.item - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometExchange + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometNativeColumnarToRow + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.item + : : +- CometBroadcastExchange + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim : +- CometSort : +- CometExchange : +- CometProject @@ -163,28 +160,29 @@ CometNativeColumnarToRow +- CometProject +- CometSortMergeJoin :- CometSort - : +- CometColumnarExchange - : +- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Filter - : : : +- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : +- ReusedSubquery - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.item - : +- BroadcastExchange - : +- CometNativeColumnarToRow - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometExchange + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.web_sales + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometNativeColumnarToRow + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.item + : +- CometBroadcastExchange + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.date_dim +- CometSort +- CometExchange +- CometProject +- CometFilter +- CometNativeScan parquet spark_catalog.default.web_returns -Comet accelerated 111 out of 167 eligible operators (66%). Final plan contains 21 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 167 out of 179 eligible operators (93%). Final plan contains 7 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q77a.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q77a.native_datafusion/extended.txt index 178b946ecb..24407fc0a2 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q77a.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q77a.native_datafusion/extended.txt @@ -9,132 +9,139 @@ CometNativeColumnarToRow : +- CometColumnarExchange : +- HashAggregate : +- Union + : :- CometNativeColumnarToRow + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometHashAggregate + : : : +- CometExchange + : : : +- CometHashAggregate + : : : +- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometNativeColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.store + : : +- CometBroadcastExchange + : : +- CometHashAggregate + : : +- CometExchange + : : +- CometHashAggregate + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.store_returns + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometNativeColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.store : :- Project - : : +- BroadcastHashJoin - : : :- HashAggregate - : : : +- CometNativeColumnarToRow - : : : +- CometColumnarExchange - : : : +- HashAggregate - : : : +- Project - : : : +- BroadcastHashJoin - : : : :- Project - : : : : +- BroadcastHashJoin - : : : : :- Filter - : : : : : +- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.store - : : +- BroadcastExchange - : : +- HashAggregate - : : +- CometNativeColumnarToRow - : : +- CometColumnarExchange - : : +- HashAggregate - : : +- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- Filter - : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.store_returns [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- ReusedSubquery - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.store - : :- Project - : : +- BroadcastNestedLoopJoin + : : +- BroadcastNestedLoopJoin [COMET: BroadcastNestedLoopJoin is not supported] : : :- BroadcastExchange - : : : +- HashAggregate - : : : +- CometNativeColumnarToRow - : : : +- CometColumnarExchange - : : : +- HashAggregate - : : : +- Project - : : : +- BroadcastHashJoin - : : : :- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- ReusedSubquery - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- HashAggregate - : : +- CometNativeColumnarToRow - : : +- CometColumnarExchange - : : +- HashAggregate - : : +- Project - : : +- BroadcastHashJoin - : : :- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.catalog_returns [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : +- ReusedSubquery - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- Project - : +- BroadcastHashJoin - : :- HashAggregate - : : +- CometNativeColumnarToRow - : : +- CometColumnarExchange - : : +- HashAggregate - : : +- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- Filter - : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- ReusedSubquery - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.web_page - : +- BroadcastExchange - : +- HashAggregate - : +- CometNativeColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Filter - : : : +- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.web_returns [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : +- ReusedSubquery - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- CometNativeColumnarToRow - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.web_page + : : : +- CometNativeColumnarToRow + : : : +- CometHashAggregate + : : : +- CometExchange + : : : +- CometHashAggregate + : : : +- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometNativeColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometNativeColumnarToRow + : : +- CometHashAggregate + : : +- CometExchange + : : +- CometHashAggregate + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometNativeScan parquet spark_catalog.default.catalog_returns + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometNativeColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometNativeColumnarToRow + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometHashAggregate + : : +- CometExchange + : : +- CometHashAggregate + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.web_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometNativeColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.web_page + : +- CometBroadcastExchange + : +- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.web_returns + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometNativeColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.web_page :- HashAggregate : +- CometNativeColumnarToRow : +- CometColumnarExchange @@ -144,132 +151,139 @@ CometNativeColumnarToRow : +- CometColumnarExchange : +- HashAggregate : +- Union + : :- CometNativeColumnarToRow + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometHashAggregate + : : : +- CometExchange + : : : +- CometHashAggregate + : : : +- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometNativeColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.store + : : +- CometBroadcastExchange + : : +- CometHashAggregate + : : +- CometExchange + : : +- CometHashAggregate + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.store_returns + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometNativeColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.store : :- Project - : : +- BroadcastHashJoin - : : :- HashAggregate - : : : +- CometNativeColumnarToRow - : : : +- CometColumnarExchange - : : : +- HashAggregate - : : : +- Project - : : : +- BroadcastHashJoin - : : : :- Project - : : : : +- BroadcastHashJoin - : : : : :- Filter - : : : : : +- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.store - : : +- BroadcastExchange - : : +- HashAggregate - : : +- CometNativeColumnarToRow - : : +- CometColumnarExchange - : : +- HashAggregate - : : +- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- Filter - : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.store_returns [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- ReusedSubquery - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.store - : :- Project - : : +- BroadcastNestedLoopJoin + : : +- BroadcastNestedLoopJoin [COMET: BroadcastNestedLoopJoin is not supported] : : :- BroadcastExchange - : : : +- HashAggregate - : : : +- CometNativeColumnarToRow - : : : +- CometColumnarExchange - : : : +- HashAggregate - : : : +- Project - : : : +- BroadcastHashJoin - : : : :- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- ReusedSubquery - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- HashAggregate - : : +- CometNativeColumnarToRow - : : +- CometColumnarExchange - : : +- HashAggregate - : : +- Project - : : +- BroadcastHashJoin - : : :- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.catalog_returns [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : +- ReusedSubquery - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- Project - : +- BroadcastHashJoin - : :- HashAggregate - : : +- CometNativeColumnarToRow - : : +- CometColumnarExchange - : : +- HashAggregate - : : +- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- Filter - : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- ReusedSubquery - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.web_page - : +- BroadcastExchange - : +- HashAggregate - : +- CometNativeColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Filter - : : : +- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.web_returns [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : +- ReusedSubquery - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- CometNativeColumnarToRow - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.web_page + : : : +- CometNativeColumnarToRow + : : : +- CometHashAggregate + : : : +- CometExchange + : : : +- CometHashAggregate + : : : +- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometNativeColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometNativeColumnarToRow + : : +- CometHashAggregate + : : +- CometExchange + : : +- CometHashAggregate + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometNativeScan parquet spark_catalog.default.catalog_returns + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometNativeColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometNativeColumnarToRow + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometHashAggregate + : : +- CometExchange + : : +- CometHashAggregate + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.web_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometNativeColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.web_page + : +- CometBroadcastExchange + : +- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.web_returns + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometNativeColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.web_page +- HashAggregate +- CometNativeColumnarToRow +- CometColumnarExchange @@ -279,131 +293,138 @@ CometNativeColumnarToRow +- CometColumnarExchange +- HashAggregate +- Union + :- CometNativeColumnarToRow + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometHashAggregate + : : +- CometExchange + : : +- CometHashAggregate + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometNativeColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.store + : +- CometBroadcastExchange + : +- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.store_returns + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometNativeColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.store :- Project - : +- BroadcastHashJoin - : :- HashAggregate - : : +- CometNativeColumnarToRow - : : +- CometColumnarExchange - : : +- HashAggregate - : : +- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- Filter - : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.store - : +- BroadcastExchange - : +- HashAggregate - : +- CometNativeColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Filter - : : : +- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.store_returns [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : +- ReusedSubquery - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- CometNativeColumnarToRow - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.store - :- Project - : +- BroadcastNestedLoopJoin + : +- BroadcastNestedLoopJoin [COMET: BroadcastNestedLoopJoin is not supported] : :- BroadcastExchange - : : +- HashAggregate - : : +- CometNativeColumnarToRow - : : +- CometColumnarExchange - : : +- HashAggregate - : : +- Project - : : +- BroadcastHashJoin - : : :- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : +- ReusedSubquery - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- HashAggregate - : +- CometNativeColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- ColumnarToRow - : : +- Scan parquet spark_catalog.default.catalog_returns [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : +- ReusedSubquery - : +- BroadcastExchange - : +- CometNativeColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.date_dim - +- Project - +- BroadcastHashJoin - :- HashAggregate - : +- CometNativeColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Filter - : : : +- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : +- ReusedSubquery - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- CometNativeColumnarToRow - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.web_page - +- BroadcastExchange - +- HashAggregate - +- CometNativeColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- Filter - : : +- ColumnarToRow - : : +- Scan parquet spark_catalog.default.web_returns [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : +- ReusedSubquery - : +- BroadcastExchange - : +- CometNativeColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.date_dim - +- BroadcastExchange - +- CometNativeColumnarToRow - +- CometFilter - +- CometNativeScan parquet spark_catalog.default.web_page + : : +- CometNativeColumnarToRow + : : +- CometHashAggregate + : : +- CometExchange + : : +- CometHashAggregate + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometNativeColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometNativeColumnarToRow + : +- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometNativeScan parquet spark_catalog.default.catalog_returns + : : +- SubqueryBroadcast + : : +- BroadcastExchange + : : +- CometNativeColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.date_dim + +- CometNativeColumnarToRow + +- CometProject + +- CometBroadcastHashJoin + :- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.web_sales + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometNativeColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.web_page + +- CometBroadcastExchange + +- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.web_returns + : : +- SubqueryBroadcast + : : +- BroadcastExchange + : : +- CometNativeColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.date_dim + +- CometBroadcastExchange + +- CometFilter + +- CometNativeScan parquet spark_catalog.default.web_page -Comet accelerated 113 out of 332 eligible operators (34%). Final plan contains 75 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 332 out of 392 eligible operators (84%). Final plan contains 36 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q78.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q78.native_datafusion/extended.txt index af4b7000b9..ebfdcc3c62 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q78.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q78.native_datafusion/extended.txt @@ -14,15 +14,14 @@ TakeOrderedAndProject : : : +- CometFilter : : : +- CometSortMergeJoin : : : :- CometSort - : : : : +- CometColumnarExchange - : : : : +- Filter - : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometExchange + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometNativeColumnarToRow + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- CometSort : : : +- CometExchange : : : +- CometProject @@ -42,11 +41,14 @@ TakeOrderedAndProject : : +- CometFilter : : +- CometSortMergeJoin : : :- CometSort - : : : +- CometColumnarExchange - : : : +- Filter - : : : +- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : +- ReusedSubquery + : : : +- CometExchange + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.web_sales + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometNativeColumnarToRow + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- CometSort : : +- CometExchange : : +- CometProject @@ -66,11 +68,14 @@ TakeOrderedAndProject : +- CometFilter : +- CometSortMergeJoin : :- CometSort - : : +- CometColumnarExchange - : : +- Filter - : : +- ColumnarToRow - : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : +- ReusedSubquery + : : +- CometExchange + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : +- SubqueryBroadcast + : : +- BroadcastExchange + : : +- CometNativeColumnarToRow + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim : +- CometSort : +- CometExchange : +- CometProject @@ -80,4 +85,4 @@ TakeOrderedAndProject +- CometFilter +- CometNativeScan parquet spark_catalog.default.date_dim -Comet accelerated 64 out of 76 eligible operators (84%). Final plan contains 5 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 74 out of 82 eligible operators (90%). Final plan contains 4 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q80a.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q80a.native_datafusion/extended.txt index da58e863bd..ec566b42de 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q80a.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q80a.native_datafusion/extended.txt @@ -22,16 +22,15 @@ CometNativeColumnarToRow : : : : : :- CometProject : : : : : : +- CometSortMergeJoin : : : : : : :- CometSort - : : : : : : : +- CometColumnarExchange - : : : : : : : +- Filter - : : : : : : : +- ColumnarToRow - : : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : : : +- SubqueryBroadcast - : : : : : : : +- BroadcastExchange - : : : : : : : +- CometNativeColumnarToRow - : : : : : : : +- CometProject - : : : : : : : +- CometFilter - : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : : +- CometExchange + : : : : : : : +- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : : : : +- SubqueryBroadcast + : : : : : : : +- BroadcastExchange + : : : : : : : +- CometNativeColumnarToRow + : : : : : : : +- CometProject + : : : : : : : +- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : : : +- CometSort : : : : : : +- CometExchange : : : : : : +- CometProject @@ -67,11 +66,15 @@ CometNativeColumnarToRow : : : : : :- CometProject : : : : : : +- CometSortMergeJoin : : : : : : :- CometSort - : : : : : : : +- CometColumnarExchange - : : : : : : : +- Filter - : : : : : : : +- ColumnarToRow - : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : : : +- ReusedSubquery + : : : : : : : +- CometExchange + : : : : : : : +- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : : : : +- SubqueryBroadcast + : : : : : : : +- BroadcastExchange + : : : : : : : +- CometNativeColumnarToRow + : : : : : : : +- CometProject + : : : : : : : +- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : : : +- CometSort : : : : : : +- CometExchange : : : : : : +- CometProject @@ -107,11 +110,15 @@ CometNativeColumnarToRow : : : : :- CometProject : : : : : +- CometSortMergeJoin : : : : : :- CometSort - : : : : : : +- CometColumnarExchange - : : : : : : +- Filter - : : : : : : +- ColumnarToRow - : : : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : : +- ReusedSubquery + : : : : : : +- CometExchange + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.web_sales + : : : : : : +- SubqueryBroadcast + : : : : : : +- BroadcastExchange + : : : : : : +- CometNativeColumnarToRow + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : : +- CometSort : : : : : +- CometExchange : : : : : +- CometProject @@ -154,16 +161,15 @@ CometNativeColumnarToRow : : : : : :- CometProject : : : : : : +- CometSortMergeJoin : : : : : : :- CometSort - : : : : : : : +- CometColumnarExchange - : : : : : : : +- Filter - : : : : : : : +- ColumnarToRow - : : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : : : +- SubqueryBroadcast - : : : : : : : +- BroadcastExchange - : : : : : : : +- CometNativeColumnarToRow - : : : : : : : +- CometProject - : : : : : : : +- CometFilter - : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : : +- CometExchange + : : : : : : : +- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : : : : +- SubqueryBroadcast + : : : : : : : +- BroadcastExchange + : : : : : : : +- CometNativeColumnarToRow + : : : : : : : +- CometProject + : : : : : : : +- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : : : +- CometSort : : : : : : +- CometExchange : : : : : : +- CometProject @@ -199,11 +205,15 @@ CometNativeColumnarToRow : : : : : :- CometProject : : : : : : +- CometSortMergeJoin : : : : : : :- CometSort - : : : : : : : +- CometColumnarExchange - : : : : : : : +- Filter - : : : : : : : +- ColumnarToRow - : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : : : +- ReusedSubquery + : : : : : : : +- CometExchange + : : : : : : : +- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : : : : +- SubqueryBroadcast + : : : : : : : +- BroadcastExchange + : : : : : : : +- CometNativeColumnarToRow + : : : : : : : +- CometProject + : : : : : : : +- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : : : +- CometSort : : : : : : +- CometExchange : : : : : : +- CometProject @@ -239,11 +249,15 @@ CometNativeColumnarToRow : : : : :- CometProject : : : : : +- CometSortMergeJoin : : : : : :- CometSort - : : : : : : +- CometColumnarExchange - : : : : : : +- Filter - : : : : : : +- ColumnarToRow - : : : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : : +- ReusedSubquery + : : : : : : +- CometExchange + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.web_sales + : : : : : : +- SubqueryBroadcast + : : : : : : +- BroadcastExchange + : : : : : : +- CometNativeColumnarToRow + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : : +- CometSort : : : : : +- CometExchange : : : : : +- CometProject @@ -286,16 +300,15 @@ CometNativeColumnarToRow : : : : :- CometProject : : : : : +- CometSortMergeJoin : : : : : :- CometSort - : : : : : : +- CometColumnarExchange - : : : : : : +- Filter - : : : : : : +- ColumnarToRow - : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometNativeColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : +- CometExchange + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : : : +- SubqueryBroadcast + : : : : : : +- BroadcastExchange + : : : : : : +- CometNativeColumnarToRow + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : : +- CometSort : : : : : +- CometExchange : : : : : +- CometProject @@ -331,11 +344,15 @@ CometNativeColumnarToRow : : : : :- CometProject : : : : : +- CometSortMergeJoin : : : : : :- CometSort - : : : : : : +- CometColumnarExchange - : : : : : : +- Filter - : : : : : : +- ColumnarToRow - : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : : +- ReusedSubquery + : : : : : : +- CometExchange + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : : : +- SubqueryBroadcast + : : : : : : +- BroadcastExchange + : : : : : : +- CometNativeColumnarToRow + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : : +- CometSort : : : : : +- CometExchange : : : : : +- CometProject @@ -371,11 +388,15 @@ CometNativeColumnarToRow : : : :- CometProject : : : : +- CometSortMergeJoin : : : : :- CometSort - : : : : : +- CometColumnarExchange - : : : : : +- Filter - : : : : : +- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : +- ReusedSubquery + : : : : : +- CometExchange + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.web_sales + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometNativeColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : +- CometSort : : : : +- CometExchange : : : : +- CometProject @@ -398,4 +419,4 @@ CometNativeColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.promotion -Comet accelerated 356 out of 386 eligible operators (92%). Final plan contains 13 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 392 out of 410 eligible operators (95%). Final plan contains 10 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q86a.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q86a.native_datafusion/extended.txt index d56e25c3af..c1df720935 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q86a.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q86a.native_datafusion/extended.txt @@ -5,97 +5,83 @@ TakeOrderedAndProject +- CometSort +- CometExchange +- CometHashAggregate - +- CometColumnarExchange - +- HashAggregate - +- Union - :- HashAggregate - : +- CometNativeColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Filter - : : : +- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- CometNativeColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.item - :- HashAggregate - : +- CometNativeColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- HashAggregate - : +- CometNativeColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Filter - : : : +- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- CometNativeColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.item - +- HashAggregate - +- CometNativeColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- HashAggregate - +- CometNativeColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- Filter - : : +- ColumnarToRow - : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- CometNativeColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.date_dim - +- BroadcastExchange - +- CometNativeColumnarToRow - +- CometProject - +- CometFilter - +- CometNativeScan parquet spark_catalog.default.item + +- CometExchange + +- CometHashAggregate + +- CometUnion + :- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.web_sales + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometNativeColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.item + :- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.web_sales + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometNativeColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.item + +- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.web_sales + : : +- SubqueryBroadcast + : : +- BroadcastExchange + : : +- CometNativeColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.date_dim + +- CometBroadcastExchange + +- CometProject + +- CometFilter + +- CometNativeScan parquet spark_catalog.default.item -Comet accelerated 36 out of 81 eligible operators (44%). Final plan contains 18 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 72 out of 81 eligible operators (88%). Final plan contains 4 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q98.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q98.native_datafusion/extended.txt index b8d60879d2..6c2b043008 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q98.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q98.native_datafusion/extended.txt @@ -5,33 +5,29 @@ CometNativeColumnarToRow +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] +- CometNativeColumnarToRow +- CometSort - +- CometColumnarExchange - +- HashAggregate - +- CometNativeColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- Filter - : : +- ColumnarToRow - : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- CometNativeColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.item - +- BroadcastExchange - +- CometNativeColumnarToRow - +- CometProject - +- CometFilter - +- CometNativeScan parquet spark_catalog.default.date_dim + +- CometExchange + +- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : +- SubqueryBroadcast + : : +- BroadcastExchange + : : +- CometNativeColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.item + +- CometBroadcastExchange + +- CometProject + +- CometFilter + +- CometNativeScan parquet spark_catalog.default.date_dim -Comet accelerated 14 out of 28 eligible operators (50%). Final plan contains 7 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 24 out of 28 eligible operators (85%). Final plan contains 3 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q10a.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q10a.native_datafusion/extended.txt index 9091536e96..cba125f412 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q10a.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q10a.native_datafusion/extended.txt @@ -1,65 +1,66 @@ -TakeOrderedAndProject -+- HashAggregate - +- CometNativeColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- BroadcastHashJoin - : : : :- CometNativeColumnarToRow - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.customer - : : : +- BroadcastExchange - : : : +- Project - : : : +- BroadcastHashJoin - : : : :- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- Union - : : :- Project - : : : +- BroadcastHashJoin - : : : :- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- ReusedSubquery - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- Project - : : +- BroadcastHashJoin - : : :- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : +- ReusedSubquery - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- CometNativeColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.customer_address - +- BroadcastExchange - +- CometNativeColumnarToRow - +- CometProject - +- CometFilter - +- CometNativeScan parquet spark_catalog.default.customer_demographics +CometNativeColumnarToRow ++- CometTakeOrderedAndProject + +- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.customer + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometNativeScan parquet spark_catalog.default.store_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometNativeColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometUnion + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometNativeScan parquet spark_catalog.default.web_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometNativeColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometNativeColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.customer_address + +- CometBroadcastExchange + +- CometProject + +- CometFilter + +- CometNativeScan parquet spark_catalog.default.customer_demographics -Comet accelerated 21 out of 52 eligible operators (40%). Final plan contains 11 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 54 out of 60 eligible operators (90%). Final plan contains 4 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q11.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q11.native_datafusion/extended.txt index 5d218998be..51a3f5ca94 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q11.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q11.native_datafusion/extended.txt @@ -1,105 +1,98 @@ -TakeOrderedAndProject -+- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- BroadcastHashJoin - : : :- Filter - : : : +- HashAggregate - : : : +- CometNativeColumnarToRow - : : : +- CometColumnarExchange - : : : +- HashAggregate - : : : +- Project - : : : +- BroadcastHashJoin - : : : :- Project - : : : : +- BroadcastHashJoin - : : : : :- CometNativeColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.customer - : : : : +- BroadcastExchange - : : : : +- Filter - : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- HashAggregate - : : +- CometNativeColumnarToRow - : : +- CometColumnarExchange - : : +- HashAggregate - : : +- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.customer - : : : +- BroadcastExchange - : : : +- Filter - : : : +- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- Filter - : +- HashAggregate - : +- CometNativeColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.customer - : : +- BroadcastExchange - : : +- Filter - : : +- ColumnarToRow - : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : +- ReusedSubquery - : +- BroadcastExchange - : +- CometNativeColumnarToRow - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.date_dim - +- BroadcastExchange - +- HashAggregate - +- CometNativeColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- CometNativeColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.customer - : +- BroadcastExchange - : +- Filter - : +- ColumnarToRow - : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : +- ReusedSubquery - +- BroadcastExchange - +- CometNativeColumnarToRow - +- CometFilter - +- CometNativeScan parquet spark_catalog.default.date_dim +CometNativeColumnarToRow ++- CometTakeOrderedAndProject + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometHashAggregate + : : : +- CometExchange + : : : +- CometHashAggregate + : : : +- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.customer + : : : : +- CometBroadcastExchange + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometNativeColumnarToRow + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometHashAggregate + : : +- CometExchange + : : +- CometHashAggregate + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.customer + : : : +- CometBroadcastExchange + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometNativeColumnarToRow + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometFilter + : +- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.customer + : : +- CometBroadcastExchange + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.web_sales + : : +- SubqueryBroadcast + : : +- BroadcastExchange + : : +- CometNativeColumnarToRow + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.date_dim + +- CometBroadcastExchange + +- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.customer + : +- CometBroadcastExchange + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.web_sales + : +- SubqueryBroadcast + : +- BroadcastExchange + : +- CometNativeColumnarToRow + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.date_dim + +- CometBroadcastExchange + +- CometFilter + +- CometNativeScan parquet spark_catalog.default.date_dim -Comet accelerated 28 out of 85 eligible operators (32%). Final plan contains 18 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 83 out of 91 eligible operators (91%). Final plan contains 5 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q12.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q12.native_datafusion/extended.txt index 15648408fc..a8c2531a38 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q12.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q12.native_datafusion/extended.txt @@ -3,33 +3,29 @@ TakeOrderedAndProject +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] +- CometNativeColumnarToRow +- CometSort - +- CometColumnarExchange - +- HashAggregate - +- CometNativeColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- Filter - : : +- ColumnarToRow - : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- CometNativeColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.item - +- BroadcastExchange - +- CometNativeColumnarToRow - +- CometProject - +- CometFilter - +- CometNativeScan parquet spark_catalog.default.date_dim + +- CometExchange + +- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.web_sales + : : +- SubqueryBroadcast + : : +- BroadcastExchange + : : +- CometNativeColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.item + +- CometBroadcastExchange + +- CometProject + +- CometFilter + +- CometNativeScan parquet spark_catalog.default.date_dim -Comet accelerated 12 out of 27 eligible operators (44%). Final plan contains 6 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 22 out of 27 eligible operators (81%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14.native_datafusion/extended.txt index 7921f2fa61..50139381f2 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14.native_datafusion/extended.txt @@ -1,404 +1,394 @@ -TakeOrderedAndProject -+- BroadcastHashJoin - :- Filter - : : +- Subquery - : : +- HashAggregate - : : +- CometNativeColumnarToRow - : : +- CometColumnarExchange - : : +- HashAggregate - : : +- Union - : : :- Project - : : : +- BroadcastHashJoin - : : : :- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- ReusedSubquery - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : :- Project - : : : +- BroadcastHashJoin - : : : :- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- ReusedSubquery - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- Project - : : +- BroadcastHashJoin - : : :- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : +- ReusedSubquery - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- HashAggregate - : +- CometNativeColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- BroadcastHashJoin - : : : :- Filter - : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : : +- Subquery - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- Project - : : : +- BroadcastHashJoin - : : : :- CometNativeColumnarToRow - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : +- BroadcastExchange - : : : +- BroadcastHashJoin - : : : :- CometNativeColumnarToRow - : : : : +- CometHashAggregate - : : : : +- CometColumnarExchange - : : : : +- HashAggregate - : : : : +- Project - : : : : +- BroadcastHashJoin - : : : : :- Project - : : : : : +- BroadcastHashJoin - : : : : : :- Filter - : : : : : : +- ColumnarToRow - : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometNativeColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : : +- BroadcastExchange - : : : : : +- BroadcastHashJoin - : : : : : :- CometNativeColumnarToRow - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : : +- BroadcastExchange - : : : : : +- Project - : : : : : +- BroadcastHashJoin - : : : : : :- Project - : : : : : : +- BroadcastHashJoin - : : : : : : :- Filter - : : : : : : : +- ColumnarToRow - : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : : : +- ReusedSubquery - : : : : : : +- BroadcastExchange - : : : : : : +- CometNativeColumnarToRow - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : : +- BroadcastExchange - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- Project - : : : +- BroadcastHashJoin - : : : :- Project - : : : : +- BroadcastHashJoin - : : : : :- Filter - : : : : : +- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : +- ReusedSubquery - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- BroadcastHashJoin - : : :- CometNativeColumnarToRow - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.item - : : +- BroadcastExchange - : : +- Project - : : +- BroadcastHashJoin - : : :- CometNativeColumnarToRow - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.item - : : +- BroadcastExchange - : : +- BroadcastHashJoin - : : :- CometNativeColumnarToRow - : : : +- CometHashAggregate - : : : +- CometColumnarExchange - : : : +- HashAggregate - : : : +- Project - : : : +- BroadcastHashJoin - : : : :- Project - : : : : +- BroadcastHashJoin - : : : : :- Filter - : : : : : +- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : +- BroadcastExchange - : : : : +- BroadcastHashJoin - : : : : :- CometNativeColumnarToRow - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : +- BroadcastExchange - : : : : +- Project - : : : : +- BroadcastHashJoin - : : : : :- Project - : : : : : +- BroadcastHashJoin - : : : : : :- Filter - : : : : : : +- ColumnarToRow - : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : : +- ReusedSubquery - : : : : : +- BroadcastExchange - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- Filter - : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- ReusedSubquery - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.item - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- CometNativeColumnarToRow - : +- CometProject - : +- CometFilter - : : +- Subquery - : : +- CometNativeColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- CometNativeScan parquet spark_catalog.default.date_dim - +- BroadcastExchange - +- Filter - : +- ReusedSubquery - +- HashAggregate - +- CometNativeColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- BroadcastHashJoin - : : :- Filter - : : : +- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : : +- Subquery - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- Project - : : +- BroadcastHashJoin - : : :- CometNativeColumnarToRow - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.item - : : +- BroadcastExchange - : : +- BroadcastHashJoin - : : :- CometNativeColumnarToRow - : : : +- CometHashAggregate - : : : +- CometColumnarExchange - : : : +- HashAggregate - : : : +- Project - : : : +- BroadcastHashJoin - : : : :- Project - : : : : +- BroadcastHashJoin - : : : : :- Filter - : : : : : +- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : +- BroadcastExchange - : : : : +- BroadcastHashJoin - : : : : :- CometNativeColumnarToRow - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : +- BroadcastExchange - : : : : +- Project - : : : : +- BroadcastHashJoin - : : : : :- Project - : : : : : +- BroadcastHashJoin - : : : : : :- Filter - : : : : : : +- ColumnarToRow - : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : : +- ReusedSubquery - : : : : : +- BroadcastExchange - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- Filter - : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- ReusedSubquery - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.item - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- BroadcastHashJoin - : :- CometNativeColumnarToRow - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.item - : +- BroadcastExchange - : +- Project - : +- BroadcastHashJoin - : :- CometNativeColumnarToRow - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.item - : +- BroadcastExchange - : +- BroadcastHashJoin - : :- CometNativeColumnarToRow - : : +- CometHashAggregate - : : +- CometColumnarExchange - : : +- HashAggregate - : : +- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- Filter - : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- BroadcastHashJoin - : : : :- CometNativeColumnarToRow - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : +- BroadcastExchange - : : : +- Project - : : : +- BroadcastHashJoin - : : : :- Project - : : : : +- BroadcastHashJoin - : : : : :- Filter - : : : : : +- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : +- ReusedSubquery - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Filter - : : : +- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : +- ReusedSubquery - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.item - : +- BroadcastExchange - : +- CometNativeColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.date_dim - +- BroadcastExchange - +- CometNativeColumnarToRow - +- CometProject - +- CometFilter - : +- Subquery - : +- CometNativeColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.date_dim - +- CometNativeScan parquet spark_catalog.default.date_dim +CometNativeColumnarToRow ++- CometTakeOrderedAndProject + +- CometBroadcastHashJoin + :- CometFilter + : : +- Subquery + : : +- CometNativeColumnarToRow + : : +- CometHashAggregate + : : +- CometExchange + : : +- CometHashAggregate + : : +- CometUnion + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometNativeScan parquet spark_catalog.default.store_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometNativeColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometNativeColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometNativeScan parquet spark_catalog.default.web_sales + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometNativeColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometNativeColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : : +- Subquery + : : : : : +- CometNativeColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : +- CometBroadcastExchange + : : : +- CometBroadcastHashJoin + : : : :- CometHashAggregate + : : : : +- CometExchange + : : : : +- CometHashAggregate + : : : : +- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometProject + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : : : +- SubqueryBroadcast + : : : : : : +- BroadcastExchange + : : : : : : +- CometNativeColumnarToRow + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- CometBroadcastExchange + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : : : +- CometBroadcastExchange + : : : : : +- CometProject + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometProject + : : : : : : +- CometBroadcastHashJoin + : : : : : : :- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : : : : +- SubqueryBroadcast + : : : : : : : +- BroadcastExchange + : : : : : : : +- CometNativeColumnarToRow + : : : : : : : +- CometProject + : : : : : : : +- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : +- CometBroadcastExchange + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : : : +- CometBroadcastExchange + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.web_sales + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometNativeColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.item + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.item + : : +- CometBroadcastExchange + : : +- CometBroadcastHashJoin + : : :- CometHashAggregate + : : : +- CometExchange + : : : +- CometHashAggregate + : : : +- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometNativeColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometBroadcastHashJoin + : : : : :- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometProject + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : : : +- SubqueryBroadcast + : : : : : : +- BroadcastExchange + : : : : : : +- CometNativeColumnarToRow + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- CometBroadcastExchange + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.web_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometNativeColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.item + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : : +- ReusedSubquery + : +- CometNativeScan parquet spark_catalog.default.date_dim + +- CometBroadcastExchange + +- CometFilter + : +- ReusedSubquery + +- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometNativeColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : : +- Subquery + : : : : +- CometNativeColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.item + : : +- CometBroadcastExchange + : : +- CometBroadcastHashJoin + : : :- CometHashAggregate + : : : +- CometExchange + : : : +- CometHashAggregate + : : : +- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometNativeColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometBroadcastHashJoin + : : : : :- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometProject + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : : : +- SubqueryBroadcast + : : : : : : +- BroadcastExchange + : : : : : : +- CometNativeColumnarToRow + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- CometBroadcastExchange + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.web_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometNativeColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.item + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometBroadcastHashJoin + : :- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.item + : +- CometBroadcastExchange + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.item + : +- CometBroadcastExchange + : +- CometBroadcastHashJoin + : :- CometHashAggregate + : : +- CometExchange + : : +- CometHashAggregate + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometNativeColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometNativeColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.web_sales + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometNativeColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.item + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.date_dim + +- CometBroadcastExchange + +- CometProject + +- CometFilter + : +- ReusedSubquery + +- CometNativeScan parquet spark_catalog.default.date_dim -Comet accelerated 128 out of 333 eligible operators (38%). Final plan contains 69 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 331 out of 371 eligible operators (89%). Final plan contains 21 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14a.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14a.native_datafusion/extended.txt index 0bc07e401e..636e5ce377 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14a.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14a.native_datafusion/extended.txt @@ -1,2779 +1,2750 @@ CometNativeColumnarToRow +- CometTakeOrderedAndProject +- CometHashAggregate - +- CometColumnarExchange - +- HashAggregate - +- Union - :- HashAggregate - : +- CometNativeColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Union - : :- Filter - : : : +- Subquery - : : : +- HashAggregate - : : : +- CometNativeColumnarToRow - : : : +- CometColumnarExchange - : : : +- HashAggregate - : : : +- Union - : : : :- Project - : : : : +- BroadcastHashJoin - : : : : :- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : +- ReusedSubquery - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : :- Project - : : : : +- BroadcastHashJoin - : : : : :- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- Project - : : : +- BroadcastHashJoin - : : : :- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- ReusedSubquery - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- HashAggregate - : : +- CometNativeColumnarToRow - : : +- CometColumnarExchange - : : +- HashAggregate - : : +- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- BroadcastHashJoin - : : : : :- Filter - : : : : : +- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : +- BroadcastExchange - : : : : +- Project - : : : : +- BroadcastHashJoin - : : : : :- CometNativeColumnarToRow - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : +- BroadcastExchange - : : : : +- BroadcastHashJoin - : : : : :- CometNativeColumnarToRow - : : : : : +- CometHashAggregate - : : : : : +- CometColumnarExchange - : : : : : +- HashAggregate - : : : : : +- Project - : : : : : +- BroadcastHashJoin - : : : : : :- Project - : : : : : : +- BroadcastHashJoin - : : : : : : :- Filter - : : : : : : : +- ColumnarToRow - : : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : : : +- SubqueryBroadcast - : : : : : : : +- BroadcastExchange - : : : : : : : +- CometNativeColumnarToRow - : : : : : : : +- CometProject - : : : : : : : +- CometFilter - : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : : : +- BroadcastExchange - : : : : : : +- BroadcastHashJoin - : : : : : : :- CometNativeColumnarToRow - : : : : : : : +- CometFilter - : : : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : : : +- BroadcastExchange - : : : : : : +- Project - : : : : : : +- BroadcastHashJoin - : : : : : : :- Project - : : : : : : : +- BroadcastHashJoin - : : : : : : : :- Filter - : : : : : : : : +- ColumnarToRow - : : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : : : : +- ReusedSubquery - : : : : : : : +- BroadcastExchange - : : : : : : : +- CometNativeColumnarToRow - : : : : : : : +- CometFilter - : : : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : : : +- BroadcastExchange - : : : : : : +- CometNativeColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : : +- BroadcastExchange - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : +- BroadcastExchange - : : : : +- Project - : : : : +- BroadcastHashJoin - : : : : :- Project - : : : : : +- BroadcastHashJoin - : : : : : :- Filter - : : : : : : +- ColumnarToRow - : : : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : : +- ReusedSubquery - : : : : : +- BroadcastExchange - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- BroadcastHashJoin - : : : :- CometNativeColumnarToRow - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : +- BroadcastExchange - : : : +- Project - : : : +- BroadcastHashJoin - : : : :- CometNativeColumnarToRow - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : +- BroadcastExchange - : : : +- BroadcastHashJoin - : : : :- CometNativeColumnarToRow - : : : : +- CometHashAggregate - : : : : +- CometColumnarExchange - : : : : +- HashAggregate - : : : : +- Project - : : : : +- BroadcastHashJoin - : : : : :- Project - : : : : : +- BroadcastHashJoin - : : : : : :- Filter - : : : : : : +- ColumnarToRow - : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometNativeColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : : +- BroadcastExchange - : : : : : +- BroadcastHashJoin - : : : : : :- CometNativeColumnarToRow - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : : +- BroadcastExchange - : : : : : +- Project - : : : : : +- BroadcastHashJoin - : : : : : :- Project - : : : : : : +- BroadcastHashJoin - : : : : : : :- Filter - : : : : : : : +- ColumnarToRow - : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : : : +- ReusedSubquery - : : : : : : +- BroadcastExchange - : : : : : : +- CometNativeColumnarToRow - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : : +- BroadcastExchange - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- Project - : : : +- BroadcastHashJoin - : : : :- Project - : : : : +- BroadcastHashJoin - : : : : :- Filter - : : : : : +- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : +- ReusedSubquery - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : :- Filter - : : : +- ReusedSubquery - : : +- HashAggregate - : : +- CometNativeColumnarToRow - : : +- CometColumnarExchange - : : +- HashAggregate - : : +- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- BroadcastHashJoin - : : : : :- Filter - : : : : : +- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : +- ReusedSubquery - : : : : +- BroadcastExchange - : : : : +- Project - : : : : +- BroadcastHashJoin - : : : : :- CometNativeColumnarToRow - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : +- BroadcastExchange - : : : : +- BroadcastHashJoin - : : : : :- CometNativeColumnarToRow - : : : : : +- CometHashAggregate - : : : : : +- CometColumnarExchange - : : : : : +- HashAggregate - : : : : : +- Project - : : : : : +- BroadcastHashJoin - : : : : : :- Project - : : : : : : +- BroadcastHashJoin - : : : : : : :- Filter - : : : : : : : +- ColumnarToRow - : : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : : : +- SubqueryBroadcast - : : : : : : : +- BroadcastExchange - : : : : : : : +- CometNativeColumnarToRow - : : : : : : : +- CometProject - : : : : : : : +- CometFilter - : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : : : +- BroadcastExchange - : : : : : : +- BroadcastHashJoin - : : : : : : :- CometNativeColumnarToRow - : : : : : : : +- CometFilter - : : : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : : : +- BroadcastExchange - : : : : : : +- Project - : : : : : : +- BroadcastHashJoin - : : : : : : :- Project - : : : : : : : +- BroadcastHashJoin - : : : : : : : :- Filter - : : : : : : : : +- ColumnarToRow - : : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : : : : +- ReusedSubquery - : : : : : : : +- BroadcastExchange - : : : : : : : +- CometNativeColumnarToRow - : : : : : : : +- CometFilter - : : : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : : : +- BroadcastExchange - : : : : : : +- CometNativeColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : : +- BroadcastExchange - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : +- BroadcastExchange - : : : : +- Project - : : : : +- BroadcastHashJoin - : : : : :- Project - : : : : : +- BroadcastHashJoin - : : : : : :- Filter - : : : : : : +- ColumnarToRow - : : : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : : +- ReusedSubquery - : : : : : +- BroadcastExchange - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- BroadcastHashJoin - : : : :- CometNativeColumnarToRow - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : +- BroadcastExchange - : : : +- Project - : : : +- BroadcastHashJoin - : : : :- CometNativeColumnarToRow - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : +- BroadcastExchange - : : : +- BroadcastHashJoin - : : : :- CometNativeColumnarToRow - : : : : +- CometHashAggregate - : : : : +- CometColumnarExchange - : : : : +- HashAggregate - : : : : +- Project - : : : : +- BroadcastHashJoin - : : : : :- Project - : : : : : +- BroadcastHashJoin - : : : : : :- Filter - : : : : : : +- ColumnarToRow - : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometNativeColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : : +- BroadcastExchange - : : : : : +- BroadcastHashJoin - : : : : : :- CometNativeColumnarToRow - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : : +- BroadcastExchange - : : : : : +- Project - : : : : : +- BroadcastHashJoin - : : : : : :- Project - : : : : : : +- BroadcastHashJoin - : : : : : : :- Filter - : : : : : : : +- ColumnarToRow - : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : : : +- ReusedSubquery - : : : : : : +- BroadcastExchange - : : : : : : +- CometNativeColumnarToRow - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : : +- BroadcastExchange - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- Project - : : : +- BroadcastHashJoin - : : : :- Project - : : : : +- BroadcastHashJoin - : : : : :- Filter - : : : : : +- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : +- ReusedSubquery - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- Filter - : : +- ReusedSubquery - : +- HashAggregate - : +- CometNativeColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- BroadcastHashJoin - : : : :- Filter - : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- ReusedSubquery - : : : +- BroadcastExchange - : : : +- Project - : : : +- BroadcastHashJoin - : : : :- CometNativeColumnarToRow - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : +- BroadcastExchange - : : : +- BroadcastHashJoin - : : : :- CometNativeColumnarToRow - : : : : +- CometHashAggregate - : : : : +- CometColumnarExchange - : : : : +- HashAggregate - : : : : +- Project - : : : : +- BroadcastHashJoin - : : : : :- Project - : : : : : +- BroadcastHashJoin - : : : : : :- Filter - : : : : : : +- ColumnarToRow - : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometNativeColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : : +- BroadcastExchange - : : : : : +- BroadcastHashJoin - : : : : : :- CometNativeColumnarToRow - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : : +- BroadcastExchange - : : : : : +- Project - : : : : : +- BroadcastHashJoin - : : : : : :- Project - : : : : : : +- BroadcastHashJoin - : : : : : : :- Filter - : : : : : : : +- ColumnarToRow - : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : : : +- ReusedSubquery - : : : : : : +- BroadcastExchange - : : : : : : +- CometNativeColumnarToRow - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : : +- BroadcastExchange - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- Project - : : : +- BroadcastHashJoin - : : : :- Project - : : : : +- BroadcastHashJoin - : : : : :- Filter - : : : : : +- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : +- ReusedSubquery - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- BroadcastHashJoin - : : :- CometNativeColumnarToRow - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.item - : : +- BroadcastExchange - : : +- Project - : : +- BroadcastHashJoin - : : :- CometNativeColumnarToRow - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.item - : : +- BroadcastExchange - : : +- BroadcastHashJoin - : : :- CometNativeColumnarToRow - : : : +- CometHashAggregate - : : : +- CometColumnarExchange - : : : +- HashAggregate - : : : +- Project - : : : +- BroadcastHashJoin - : : : :- Project - : : : : +- BroadcastHashJoin - : : : : :- Filter - : : : : : +- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : +- BroadcastExchange - : : : : +- BroadcastHashJoin - : : : : :- CometNativeColumnarToRow - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : +- BroadcastExchange - : : : : +- Project - : : : : +- BroadcastHashJoin - : : : : :- Project - : : : : : +- BroadcastHashJoin - : : : : : :- Filter - : : : : : : +- ColumnarToRow - : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : : +- ReusedSubquery - : : : : : +- BroadcastExchange - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- Filter - : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- ReusedSubquery - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.item - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- CometNativeColumnarToRow + +- CometExchange + +- CometHashAggregate + +- CometUnion + :- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometUnion + : :- CometFilter + : : : +- Subquery + : : : +- CometNativeColumnarToRow + : : : +- CometHashAggregate + : : : +- CometExchange + : : : +- CometHashAggregate + : : : +- CometUnion + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometNativeScan parquet spark_catalog.default.store_sales + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometNativeColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometNativeColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometNativeScan parquet spark_catalog.default.web_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometNativeColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometHashAggregate + : : +- CometExchange + : : +- CometHashAggregate + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometBroadcastHashJoin + : : : : :- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometNativeColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : : +- CometBroadcastExchange + : : : : +- CometBroadcastHashJoin + : : : : :- CometHashAggregate + : : : : : +- CometExchange + : : : : : +- CometHashAggregate + : : : : : +- CometProject + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometProject + : : : : : : +- CometBroadcastHashJoin + : : : : : : :- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : : : : +- SubqueryBroadcast + : : : : : : : +- BroadcastExchange + : : : : : : : +- CometNativeColumnarToRow + : : : : : : : +- CometProject + : : : : : : : +- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : +- CometBroadcastExchange + : : : : : : +- CometBroadcastHashJoin + : : : : : : :- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : : : : +- CometBroadcastExchange + : : : : : : +- CometProject + : : : : : : +- CometBroadcastHashJoin + : : : : : : :- CometProject + : : : : : : : +- CometBroadcastHashJoin + : : : : : : : :- CometFilter + : : : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : : : : : +- SubqueryBroadcast + : : : : : : : : +- BroadcastExchange + : : : : : : : : +- CometNativeColumnarToRow + : : : : : : : : +- CometProject + : : : : : : : : +- CometFilter + : : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : : +- CometBroadcastExchange + : : : : : : : +- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : : : : +- CometBroadcastExchange + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- CometBroadcastExchange + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometProject + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.web_sales + : : : : : : +- SubqueryBroadcast + : : : : : : +- BroadcastExchange + : : : : : : +- CometNativeColumnarToRow + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- CometBroadcastExchange + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : +- CometBroadcastExchange + : : : +- CometBroadcastHashJoin + : : : :- CometHashAggregate + : : : : +- CometExchange + : : : : +- CometHashAggregate + : : : : +- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometProject + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : : : +- SubqueryBroadcast + : : : : : : +- BroadcastExchange + : : : : : : +- CometNativeColumnarToRow + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- CometBroadcastExchange + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : : : +- CometBroadcastExchange + : : : : : +- CometProject + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometProject + : : : : : : +- CometBroadcastHashJoin + : : : : : : :- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : : : : +- SubqueryBroadcast + : : : : : : : +- BroadcastExchange + : : : : : : : +- CometNativeColumnarToRow + : : : : : : : +- CometProject + : : : : : : : +- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : +- CometBroadcastExchange + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : : : +- CometBroadcastExchange + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.web_sales + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometNativeColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : :- CometFilter + : : : +- ReusedSubquery + : : +- CometHashAggregate + : : +- CometExchange + : : +- CometHashAggregate + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometBroadcastHashJoin + : : : : :- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometNativeColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : : +- CometBroadcastExchange + : : : : +- CometBroadcastHashJoin + : : : : :- CometHashAggregate + : : : : : +- CometExchange + : : : : : +- CometHashAggregate + : : : : : +- CometProject + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometProject + : : : : : : +- CometBroadcastHashJoin + : : : : : : :- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : : : : +- SubqueryBroadcast + : : : : : : : +- BroadcastExchange + : : : : : : : +- CometNativeColumnarToRow + : : : : : : : +- CometProject + : : : : : : : +- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : +- CometBroadcastExchange + : : : : : : +- CometBroadcastHashJoin + : : : : : : :- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : : : : +- CometBroadcastExchange + : : : : : : +- CometProject + : : : : : : +- CometBroadcastHashJoin + : : : : : : :- CometProject + : : : : : : : +- CometBroadcastHashJoin + : : : : : : : :- CometFilter + : : : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : : : : : +- SubqueryBroadcast + : : : : : : : : +- BroadcastExchange + : : : : : : : : +- CometNativeColumnarToRow + : : : : : : : : +- CometProject + : : : : : : : : +- CometFilter + : : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : : +- CometBroadcastExchange + : : : : : : : +- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : : : : +- CometBroadcastExchange + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- CometBroadcastExchange + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometProject + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.web_sales + : : : : : : +- SubqueryBroadcast + : : : : : : +- BroadcastExchange + : : : : : : +- CometNativeColumnarToRow + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- CometBroadcastExchange + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : +- CometBroadcastExchange + : : : +- CometBroadcastHashJoin + : : : :- CometHashAggregate + : : : : +- CometExchange + : : : : +- CometHashAggregate + : : : : +- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometProject + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : : : +- SubqueryBroadcast + : : : : : : +- BroadcastExchange + : : : : : : +- CometNativeColumnarToRow + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- CometBroadcastExchange + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : : : +- CometBroadcastExchange + : : : : : +- CometProject + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometProject + : : : : : : +- CometBroadcastHashJoin + : : : : : : :- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : : : : +- SubqueryBroadcast + : : : : : : : +- BroadcastExchange + : : : : : : : +- CometNativeColumnarToRow + : : : : : : : +- CometProject + : : : : : : : +- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : +- CometBroadcastExchange + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : : : +- CometBroadcastExchange + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.web_sales + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometNativeColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometFilter + : : +- ReusedSubquery + : +- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.web_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometNativeColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : +- CometBroadcastExchange + : : : +- CometBroadcastHashJoin + : : : :- CometHashAggregate + : : : : +- CometExchange + : : : : +- CometHashAggregate + : : : : +- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometProject + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : : : +- SubqueryBroadcast + : : : : : : +- BroadcastExchange + : : : : : : +- CometNativeColumnarToRow + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- CometBroadcastExchange + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : : : +- CometBroadcastExchange + : : : : : +- CometProject + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometProject + : : : : : : +- CometBroadcastHashJoin + : : : : : : :- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : : : : +- SubqueryBroadcast + : : : : : : : +- BroadcastExchange + : : : : : : : +- CometNativeColumnarToRow + : : : : : : : +- CometProject + : : : : : : : +- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : +- CometBroadcastExchange + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : : : +- CometBroadcastExchange + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.web_sales + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometNativeColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.item + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.item + : : +- CometBroadcastExchange + : : +- CometBroadcastHashJoin + : : :- CometHashAggregate + : : : +- CometExchange + : : : +- CometHashAggregate + : : : +- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometNativeColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometBroadcastHashJoin + : : : : :- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometProject + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : : : +- SubqueryBroadcast + : : : : : : +- BroadcastExchange + : : : : : : +- CometNativeColumnarToRow + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- CometBroadcastExchange + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.web_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometNativeColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.item + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.date_dim + :- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometUnion + : :- CometFilter + : : : +- Subquery + : : : +- CometNativeColumnarToRow + : : : +- CometHashAggregate + : : : +- CometExchange + : : : +- CometHashAggregate + : : : +- CometUnion + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometNativeScan parquet spark_catalog.default.store_sales + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometNativeColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometNativeColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometNativeScan parquet spark_catalog.default.web_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometNativeColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometHashAggregate + : : +- CometExchange + : : +- CometHashAggregate + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometBroadcastHashJoin + : : : : :- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometNativeColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : : +- CometBroadcastExchange + : : : : +- CometBroadcastHashJoin + : : : : :- CometHashAggregate + : : : : : +- CometExchange + : : : : : +- CometHashAggregate + : : : : : +- CometProject + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometProject + : : : : : : +- CometBroadcastHashJoin + : : : : : : :- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : : : : +- SubqueryBroadcast + : : : : : : : +- BroadcastExchange + : : : : : : : +- CometNativeColumnarToRow + : : : : : : : +- CometProject + : : : : : : : +- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : +- CometBroadcastExchange + : : : : : : +- CometBroadcastHashJoin + : : : : : : :- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : : : : +- CometBroadcastExchange + : : : : : : +- CometProject + : : : : : : +- CometBroadcastHashJoin + : : : : : : :- CometProject + : : : : : : : +- CometBroadcastHashJoin + : : : : : : : :- CometFilter + : : : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : : : : : +- SubqueryBroadcast + : : : : : : : : +- BroadcastExchange + : : : : : : : : +- CometNativeColumnarToRow + : : : : : : : : +- CometProject + : : : : : : : : +- CometFilter + : : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : : +- CometBroadcastExchange + : : : : : : : +- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : : : : +- CometBroadcastExchange + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- CometBroadcastExchange + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometProject + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.web_sales + : : : : : : +- SubqueryBroadcast + : : : : : : +- BroadcastExchange + : : : : : : +- CometNativeColumnarToRow + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- CometBroadcastExchange + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : +- CometBroadcastExchange + : : : +- CometBroadcastHashJoin + : : : :- CometHashAggregate + : : : : +- CometExchange + : : : : +- CometHashAggregate + : : : : +- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometProject + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : : : +- SubqueryBroadcast + : : : : : : +- BroadcastExchange + : : : : : : +- CometNativeColumnarToRow + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- CometBroadcastExchange + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : : : +- CometBroadcastExchange + : : : : : +- CometProject + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometProject + : : : : : : +- CometBroadcastHashJoin + : : : : : : :- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : : : : +- SubqueryBroadcast + : : : : : : : +- BroadcastExchange + : : : : : : : +- CometNativeColumnarToRow + : : : : : : : +- CometProject + : : : : : : : +- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : +- CometBroadcastExchange + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : : : +- CometBroadcastExchange + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.web_sales + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometNativeColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : :- CometFilter + : : : +- ReusedSubquery + : : +- CometHashAggregate + : : +- CometExchange + : : +- CometHashAggregate + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometBroadcastHashJoin + : : : : :- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometNativeColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : : +- CometBroadcastExchange + : : : : +- CometBroadcastHashJoin + : : : : :- CometHashAggregate + : : : : : +- CometExchange + : : : : : +- CometHashAggregate + : : : : : +- CometProject + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometProject + : : : : : : +- CometBroadcastHashJoin + : : : : : : :- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : : : : +- SubqueryBroadcast + : : : : : : : +- BroadcastExchange + : : : : : : : +- CometNativeColumnarToRow + : : : : : : : +- CometProject + : : : : : : : +- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : +- CometBroadcastExchange + : : : : : : +- CometBroadcastHashJoin + : : : : : : :- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : : : : +- CometBroadcastExchange + : : : : : : +- CometProject + : : : : : : +- CometBroadcastHashJoin + : : : : : : :- CometProject + : : : : : : : +- CometBroadcastHashJoin + : : : : : : : :- CometFilter + : : : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : : : : : +- SubqueryBroadcast + : : : : : : : : +- BroadcastExchange + : : : : : : : : +- CometNativeColumnarToRow + : : : : : : : : +- CometProject + : : : : : : : : +- CometFilter + : : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : : +- CometBroadcastExchange + : : : : : : : +- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : : : : +- CometBroadcastExchange + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- CometBroadcastExchange + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometProject + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.web_sales + : : : : : : +- SubqueryBroadcast + : : : : : : +- BroadcastExchange + : : : : : : +- CometNativeColumnarToRow + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- CometBroadcastExchange + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : +- CometBroadcastExchange + : : : +- CometBroadcastHashJoin + : : : :- CometHashAggregate + : : : : +- CometExchange + : : : : +- CometHashAggregate + : : : : +- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometProject + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : : : +- SubqueryBroadcast + : : : : : : +- BroadcastExchange + : : : : : : +- CometNativeColumnarToRow + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- CometBroadcastExchange + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : : : +- CometBroadcastExchange + : : : : : +- CometProject + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometProject + : : : : : : +- CometBroadcastHashJoin + : : : : : : :- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : : : : +- SubqueryBroadcast + : : : : : : : +- BroadcastExchange + : : : : : : : +- CometNativeColumnarToRow + : : : : : : : +- CometProject + : : : : : : : +- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : +- CometBroadcastExchange + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : : : +- CometBroadcastExchange + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.web_sales + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometNativeColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometFilter + : : +- ReusedSubquery + : +- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.web_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometNativeColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : +- CometBroadcastExchange + : : : +- CometBroadcastHashJoin + : : : :- CometHashAggregate + : : : : +- CometExchange + : : : : +- CometHashAggregate + : : : : +- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometProject + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : : : +- SubqueryBroadcast + : : : : : : +- BroadcastExchange + : : : : : : +- CometNativeColumnarToRow + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- CometBroadcastExchange + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : : : +- CometBroadcastExchange + : : : : : +- CometProject + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometProject + : : : : : : +- CometBroadcastHashJoin + : : : : : : :- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : : : : +- SubqueryBroadcast + : : : : : : : +- BroadcastExchange + : : : : : : : +- CometNativeColumnarToRow + : : : : : : : +- CometProject + : : : : : : : +- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : +- CometBroadcastExchange + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : : : +- CometBroadcastExchange + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.web_sales + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometNativeColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.item + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.item + : : +- CometBroadcastExchange + : : +- CometBroadcastHashJoin + : : :- CometHashAggregate + : : : +- CometExchange + : : : +- CometHashAggregate + : : : +- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometNativeColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometBroadcastHashJoin + : : : : :- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometProject + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : : : +- SubqueryBroadcast + : : : : : : +- BroadcastExchange + : : : : : : +- CometNativeColumnarToRow + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- CometBroadcastExchange + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.web_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometNativeColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.item + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange : +- CometProject : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.date_dim - :- HashAggregate - : +- CometNativeColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- HashAggregate - : +- CometNativeColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Union - : :- Filter - : : : +- Subquery - : : : +- HashAggregate - : : : +- CometNativeColumnarToRow - : : : +- CometColumnarExchange - : : : +- HashAggregate - : : : +- Union - : : : :- Project - : : : : +- BroadcastHashJoin - : : : : :- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : +- ReusedSubquery - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : :- Project - : : : : +- BroadcastHashJoin - : : : : :- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- Project - : : : +- BroadcastHashJoin - : : : :- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- ReusedSubquery - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- HashAggregate - : : +- CometNativeColumnarToRow - : : +- CometColumnarExchange - : : +- HashAggregate - : : +- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- BroadcastHashJoin - : : : : :- Filter - : : : : : +- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : +- BroadcastExchange - : : : : +- Project - : : : : +- BroadcastHashJoin - : : : : :- CometNativeColumnarToRow - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : +- BroadcastExchange - : : : : +- BroadcastHashJoin - : : : : :- CometNativeColumnarToRow - : : : : : +- CometHashAggregate - : : : : : +- CometColumnarExchange - : : : : : +- HashAggregate - : : : : : +- Project - : : : : : +- BroadcastHashJoin - : : : : : :- Project - : : : : : : +- BroadcastHashJoin - : : : : : : :- Filter - : : : : : : : +- ColumnarToRow - : : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : : : +- SubqueryBroadcast - : : : : : : : +- BroadcastExchange - : : : : : : : +- CometNativeColumnarToRow - : : : : : : : +- CometProject - : : : : : : : +- CometFilter - : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : : : +- BroadcastExchange - : : : : : : +- BroadcastHashJoin - : : : : : : :- CometNativeColumnarToRow - : : : : : : : +- CometFilter - : : : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : : : +- BroadcastExchange - : : : : : : +- Project - : : : : : : +- BroadcastHashJoin - : : : : : : :- Project - : : : : : : : +- BroadcastHashJoin - : : : : : : : :- Filter - : : : : : : : : +- ColumnarToRow - : : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : : : : +- ReusedSubquery - : : : : : : : +- BroadcastExchange - : : : : : : : +- CometNativeColumnarToRow - : : : : : : : +- CometFilter - : : : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : : : +- BroadcastExchange - : : : : : : +- CometNativeColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : : +- BroadcastExchange - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : +- BroadcastExchange - : : : : +- Project - : : : : +- BroadcastHashJoin - : : : : :- Project - : : : : : +- BroadcastHashJoin - : : : : : :- Filter - : : : : : : +- ColumnarToRow - : : : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : : +- ReusedSubquery - : : : : : +- BroadcastExchange - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- BroadcastHashJoin - : : : :- CometNativeColumnarToRow - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : +- BroadcastExchange - : : : +- Project - : : : +- BroadcastHashJoin - : : : :- CometNativeColumnarToRow - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : +- BroadcastExchange - : : : +- BroadcastHashJoin - : : : :- CometNativeColumnarToRow - : : : : +- CometHashAggregate - : : : : +- CometColumnarExchange - : : : : +- HashAggregate - : : : : +- Project - : : : : +- BroadcastHashJoin - : : : : :- Project - : : : : : +- BroadcastHashJoin - : : : : : :- Filter - : : : : : : +- ColumnarToRow - : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometNativeColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : : +- BroadcastExchange - : : : : : +- BroadcastHashJoin - : : : : : :- CometNativeColumnarToRow - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : : +- BroadcastExchange - : : : : : +- Project - : : : : : +- BroadcastHashJoin - : : : : : :- Project - : : : : : : +- BroadcastHashJoin - : : : : : : :- Filter - : : : : : : : +- ColumnarToRow - : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : : : +- ReusedSubquery - : : : : : : +- BroadcastExchange - : : : : : : +- CometNativeColumnarToRow - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : : +- BroadcastExchange - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- Project - : : : +- BroadcastHashJoin - : : : :- Project - : : : : +- BroadcastHashJoin - : : : : :- Filter - : : : : : +- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : +- ReusedSubquery - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : :- Filter - : : : +- ReusedSubquery - : : +- HashAggregate - : : +- CometNativeColumnarToRow - : : +- CometColumnarExchange - : : +- HashAggregate - : : +- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- BroadcastHashJoin - : : : : :- Filter - : : : : : +- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : +- ReusedSubquery - : : : : +- BroadcastExchange - : : : : +- Project - : : : : +- BroadcastHashJoin - : : : : :- CometNativeColumnarToRow - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : +- BroadcastExchange - : : : : +- BroadcastHashJoin - : : : : :- CometNativeColumnarToRow - : : : : : +- CometHashAggregate - : : : : : +- CometColumnarExchange - : : : : : +- HashAggregate - : : : : : +- Project - : : : : : +- BroadcastHashJoin - : : : : : :- Project - : : : : : : +- BroadcastHashJoin - : : : : : : :- Filter - : : : : : : : +- ColumnarToRow - : : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : : : +- SubqueryBroadcast - : : : : : : : +- BroadcastExchange - : : : : : : : +- CometNativeColumnarToRow - : : : : : : : +- CometProject - : : : : : : : +- CometFilter - : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : : : +- BroadcastExchange - : : : : : : +- BroadcastHashJoin - : : : : : : :- CometNativeColumnarToRow - : : : : : : : +- CometFilter - : : : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : : : +- BroadcastExchange - : : : : : : +- Project - : : : : : : +- BroadcastHashJoin - : : : : : : :- Project - : : : : : : : +- BroadcastHashJoin - : : : : : : : :- Filter - : : : : : : : : +- ColumnarToRow - : : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : : : : +- ReusedSubquery - : : : : : : : +- BroadcastExchange - : : : : : : : +- CometNativeColumnarToRow - : : : : : : : +- CometFilter - : : : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : : : +- BroadcastExchange - : : : : : : +- CometNativeColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : : +- BroadcastExchange - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : +- BroadcastExchange - : : : : +- Project - : : : : +- BroadcastHashJoin - : : : : :- Project - : : : : : +- BroadcastHashJoin - : : : : : :- Filter - : : : : : : +- ColumnarToRow - : : : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : : +- ReusedSubquery - : : : : : +- BroadcastExchange - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- BroadcastHashJoin - : : : :- CometNativeColumnarToRow - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : +- BroadcastExchange - : : : +- Project - : : : +- BroadcastHashJoin - : : : :- CometNativeColumnarToRow - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : +- BroadcastExchange - : : : +- BroadcastHashJoin - : : : :- CometNativeColumnarToRow - : : : : +- CometHashAggregate - : : : : +- CometColumnarExchange - : : : : +- HashAggregate - : : : : +- Project - : : : : +- BroadcastHashJoin - : : : : :- Project - : : : : : +- BroadcastHashJoin - : : : : : :- Filter - : : : : : : +- ColumnarToRow - : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometNativeColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : : +- BroadcastExchange - : : : : : +- BroadcastHashJoin - : : : : : :- CometNativeColumnarToRow - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : : +- BroadcastExchange - : : : : : +- Project - : : : : : +- BroadcastHashJoin - : : : : : :- Project - : : : : : : +- BroadcastHashJoin - : : : : : : :- Filter - : : : : : : : +- ColumnarToRow - : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : : : +- ReusedSubquery - : : : : : : +- BroadcastExchange - : : : : : : +- CometNativeColumnarToRow - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : : +- BroadcastExchange - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- Project - : : : +- BroadcastHashJoin - : : : :- Project - : : : : +- BroadcastHashJoin - : : : : :- Filter - : : : : : +- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : +- ReusedSubquery - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- Filter - : : +- ReusedSubquery - : +- HashAggregate - : +- CometNativeColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- BroadcastHashJoin - : : : :- Filter - : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- ReusedSubquery - : : : +- BroadcastExchange - : : : +- Project - : : : +- BroadcastHashJoin - : : : :- CometNativeColumnarToRow - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : +- BroadcastExchange - : : : +- BroadcastHashJoin - : : : :- CometNativeColumnarToRow - : : : : +- CometHashAggregate - : : : : +- CometColumnarExchange - : : : : +- HashAggregate - : : : : +- Project - : : : : +- BroadcastHashJoin - : : : : :- Project - : : : : : +- BroadcastHashJoin - : : : : : :- Filter - : : : : : : +- ColumnarToRow - : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometNativeColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : : +- BroadcastExchange - : : : : : +- BroadcastHashJoin - : : : : : :- CometNativeColumnarToRow - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : : +- BroadcastExchange - : : : : : +- Project - : : : : : +- BroadcastHashJoin - : : : : : :- Project - : : : : : : +- BroadcastHashJoin - : : : : : : :- Filter - : : : : : : : +- ColumnarToRow - : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : : : +- ReusedSubquery - : : : : : : +- BroadcastExchange - : : : : : : +- CometNativeColumnarToRow - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : : +- BroadcastExchange - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- Project - : : : +- BroadcastHashJoin - : : : :- Project - : : : : +- BroadcastHashJoin - : : : : :- Filter - : : : : : +- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : +- ReusedSubquery - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- BroadcastHashJoin - : : :- CometNativeColumnarToRow - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.item - : : +- BroadcastExchange - : : +- Project - : : +- BroadcastHashJoin - : : :- CometNativeColumnarToRow - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.item - : : +- BroadcastExchange - : : +- BroadcastHashJoin - : : :- CometNativeColumnarToRow - : : : +- CometHashAggregate - : : : +- CometColumnarExchange - : : : +- HashAggregate - : : : +- Project - : : : +- BroadcastHashJoin - : : : :- Project - : : : : +- BroadcastHashJoin - : : : : :- Filter - : : : : : +- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : +- BroadcastExchange - : : : : +- BroadcastHashJoin - : : : : :- CometNativeColumnarToRow - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : +- BroadcastExchange - : : : : +- Project - : : : : +- BroadcastHashJoin - : : : : :- Project - : : : : : +- BroadcastHashJoin - : : : : : :- Filter - : : : : : : +- ColumnarToRow - : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : : +- ReusedSubquery - : : : : : +- BroadcastExchange - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- Filter - : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- ReusedSubquery - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.item - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- CometNativeColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.date_dim - :- HashAggregate - : +- CometNativeColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- HashAggregate - : +- CometNativeColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Union - : :- Filter - : : : +- Subquery - : : : +- HashAggregate - : : : +- CometNativeColumnarToRow - : : : +- CometColumnarExchange - : : : +- HashAggregate - : : : +- Union - : : : :- Project - : : : : +- BroadcastHashJoin - : : : : :- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : +- ReusedSubquery - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : :- Project - : : : : +- BroadcastHashJoin - : : : : :- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- Project - : : : +- BroadcastHashJoin - : : : :- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- ReusedSubquery - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- HashAggregate - : : +- CometNativeColumnarToRow - : : +- CometColumnarExchange - : : +- HashAggregate - : : +- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- BroadcastHashJoin - : : : : :- Filter - : : : : : +- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : +- BroadcastExchange - : : : : +- Project - : : : : +- BroadcastHashJoin - : : : : :- CometNativeColumnarToRow - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : +- BroadcastExchange - : : : : +- BroadcastHashJoin - : : : : :- CometNativeColumnarToRow - : : : : : +- CometHashAggregate - : : : : : +- CometColumnarExchange - : : : : : +- HashAggregate - : : : : : +- Project - : : : : : +- BroadcastHashJoin - : : : : : :- Project - : : : : : : +- BroadcastHashJoin - : : : : : : :- Filter - : : : : : : : +- ColumnarToRow - : : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : : : +- SubqueryBroadcast - : : : : : : : +- BroadcastExchange - : : : : : : : +- CometNativeColumnarToRow - : : : : : : : +- CometProject - : : : : : : : +- CometFilter - : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : : : +- BroadcastExchange - : : : : : : +- BroadcastHashJoin - : : : : : : :- CometNativeColumnarToRow - : : : : : : : +- CometFilter - : : : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : : : +- BroadcastExchange - : : : : : : +- Project - : : : : : : +- BroadcastHashJoin - : : : : : : :- Project - : : : : : : : +- BroadcastHashJoin - : : : : : : : :- Filter - : : : : : : : : +- ColumnarToRow - : : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : : : : +- ReusedSubquery - : : : : : : : +- BroadcastExchange - : : : : : : : +- CometNativeColumnarToRow - : : : : : : : +- CometFilter - : : : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : : : +- BroadcastExchange - : : : : : : +- CometNativeColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : : +- BroadcastExchange - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : +- BroadcastExchange - : : : : +- Project - : : : : +- BroadcastHashJoin - : : : : :- Project - : : : : : +- BroadcastHashJoin - : : : : : :- Filter - : : : : : : +- ColumnarToRow - : : : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : : +- ReusedSubquery - : : : : : +- BroadcastExchange - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- BroadcastHashJoin - : : : :- CometNativeColumnarToRow - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : +- BroadcastExchange - : : : +- Project - : : : +- BroadcastHashJoin - : : : :- CometNativeColumnarToRow - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : +- BroadcastExchange - : : : +- BroadcastHashJoin - : : : :- CometNativeColumnarToRow - : : : : +- CometHashAggregate - : : : : +- CometColumnarExchange - : : : : +- HashAggregate - : : : : +- Project - : : : : +- BroadcastHashJoin - : : : : :- Project - : : : : : +- BroadcastHashJoin - : : : : : :- Filter - : : : : : : +- ColumnarToRow - : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometNativeColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : : +- BroadcastExchange - : : : : : +- BroadcastHashJoin - : : : : : :- CometNativeColumnarToRow - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : : +- BroadcastExchange - : : : : : +- Project - : : : : : +- BroadcastHashJoin - : : : : : :- Project - : : : : : : +- BroadcastHashJoin - : : : : : : :- Filter - : : : : : : : +- ColumnarToRow - : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : : : +- ReusedSubquery - : : : : : : +- BroadcastExchange - : : : : : : +- CometNativeColumnarToRow - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : : +- BroadcastExchange - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- Project - : : : +- BroadcastHashJoin - : : : :- Project - : : : : +- BroadcastHashJoin - : : : : :- Filter - : : : : : +- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : +- ReusedSubquery - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : :- Filter - : : : +- ReusedSubquery - : : +- HashAggregate - : : +- CometNativeColumnarToRow - : : +- CometColumnarExchange - : : +- HashAggregate - : : +- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- BroadcastHashJoin - : : : : :- Filter - : : : : : +- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : +- ReusedSubquery - : : : : +- BroadcastExchange - : : : : +- Project - : : : : +- BroadcastHashJoin - : : : : :- CometNativeColumnarToRow - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : +- BroadcastExchange - : : : : +- BroadcastHashJoin - : : : : :- CometNativeColumnarToRow - : : : : : +- CometHashAggregate - : : : : : +- CometColumnarExchange - : : : : : +- HashAggregate - : : : : : +- Project - : : : : : +- BroadcastHashJoin - : : : : : :- Project - : : : : : : +- BroadcastHashJoin - : : : : : : :- Filter - : : : : : : : +- ColumnarToRow - : : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : : : +- SubqueryBroadcast - : : : : : : : +- BroadcastExchange - : : : : : : : +- CometNativeColumnarToRow - : : : : : : : +- CometProject - : : : : : : : +- CometFilter - : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : : : +- BroadcastExchange - : : : : : : +- BroadcastHashJoin - : : : : : : :- CometNativeColumnarToRow - : : : : : : : +- CometFilter - : : : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : : : +- BroadcastExchange - : : : : : : +- Project - : : : : : : +- BroadcastHashJoin - : : : : : : :- Project - : : : : : : : +- BroadcastHashJoin - : : : : : : : :- Filter - : : : : : : : : +- ColumnarToRow - : : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : : : : +- ReusedSubquery - : : : : : : : +- BroadcastExchange - : : : : : : : +- CometNativeColumnarToRow - : : : : : : : +- CometFilter - : : : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : : : +- BroadcastExchange - : : : : : : +- CometNativeColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : : +- BroadcastExchange - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : +- BroadcastExchange - : : : : +- Project - : : : : +- BroadcastHashJoin - : : : : :- Project - : : : : : +- BroadcastHashJoin - : : : : : :- Filter - : : : : : : +- ColumnarToRow - : : : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : : +- ReusedSubquery - : : : : : +- BroadcastExchange - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- BroadcastHashJoin - : : : :- CometNativeColumnarToRow - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : +- BroadcastExchange - : : : +- Project - : : : +- BroadcastHashJoin - : : : :- CometNativeColumnarToRow - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : +- BroadcastExchange - : : : +- BroadcastHashJoin - : : : :- CometNativeColumnarToRow - : : : : +- CometHashAggregate - : : : : +- CometColumnarExchange - : : : : +- HashAggregate - : : : : +- Project - : : : : +- BroadcastHashJoin - : : : : :- Project - : : : : : +- BroadcastHashJoin - : : : : : :- Filter - : : : : : : +- ColumnarToRow - : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometNativeColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : : +- BroadcastExchange - : : : : : +- BroadcastHashJoin - : : : : : :- CometNativeColumnarToRow - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : : +- BroadcastExchange - : : : : : +- Project - : : : : : +- BroadcastHashJoin - : : : : : :- Project - : : : : : : +- BroadcastHashJoin - : : : : : : :- Filter - : : : : : : : +- ColumnarToRow - : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : : : +- ReusedSubquery - : : : : : : +- BroadcastExchange - : : : : : : +- CometNativeColumnarToRow - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : : +- BroadcastExchange - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- Project - : : : +- BroadcastHashJoin - : : : :- Project - : : : : +- BroadcastHashJoin - : : : : :- Filter - : : : : : +- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : +- ReusedSubquery - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- Filter - : : +- ReusedSubquery - : +- HashAggregate - : +- CometNativeColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- BroadcastHashJoin - : : : :- Filter - : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- ReusedSubquery - : : : +- BroadcastExchange - : : : +- Project - : : : +- BroadcastHashJoin - : : : :- CometNativeColumnarToRow - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : +- BroadcastExchange - : : : +- BroadcastHashJoin - : : : :- CometNativeColumnarToRow - : : : : +- CometHashAggregate - : : : : +- CometColumnarExchange - : : : : +- HashAggregate - : : : : +- Project - : : : : +- BroadcastHashJoin - : : : : :- Project - : : : : : +- BroadcastHashJoin - : : : : : :- Filter - : : : : : : +- ColumnarToRow - : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometNativeColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : : +- BroadcastExchange - : : : : : +- BroadcastHashJoin - : : : : : :- CometNativeColumnarToRow - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : : +- BroadcastExchange - : : : : : +- Project - : : : : : +- BroadcastHashJoin - : : : : : :- Project - : : : : : : +- BroadcastHashJoin - : : : : : : :- Filter - : : : : : : : +- ColumnarToRow - : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : : : +- ReusedSubquery - : : : : : : +- BroadcastExchange - : : : : : : +- CometNativeColumnarToRow - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : : +- BroadcastExchange - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- Project - : : : +- BroadcastHashJoin - : : : :- Project - : : : : +- BroadcastHashJoin - : : : : :- Filter - : : : : : +- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : +- ReusedSubquery - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- BroadcastHashJoin - : : :- CometNativeColumnarToRow - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.item - : : +- BroadcastExchange - : : +- Project - : : +- BroadcastHashJoin - : : :- CometNativeColumnarToRow - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.item - : : +- BroadcastExchange - : : +- BroadcastHashJoin - : : :- CometNativeColumnarToRow - : : : +- CometHashAggregate - : : : +- CometColumnarExchange - : : : +- HashAggregate - : : : +- Project - : : : +- BroadcastHashJoin - : : : :- Project - : : : : +- BroadcastHashJoin - : : : : :- Filter - : : : : : +- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : +- BroadcastExchange - : : : : +- BroadcastHashJoin - : : : : :- CometNativeColumnarToRow - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : +- BroadcastExchange - : : : : +- Project - : : : : +- BroadcastHashJoin - : : : : :- Project - : : : : : +- BroadcastHashJoin - : : : : : :- Filter - : : : : : : +- ColumnarToRow - : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : : +- ReusedSubquery - : : : : : +- BroadcastExchange - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- Filter - : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- ReusedSubquery - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.item - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- CometNativeColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.date_dim - :- HashAggregate - : +- CometNativeColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- HashAggregate - : +- CometNativeColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Union - : :- Filter - : : : +- Subquery - : : : +- HashAggregate - : : : +- CometNativeColumnarToRow - : : : +- CometColumnarExchange - : : : +- HashAggregate - : : : +- Union - : : : :- Project - : : : : +- BroadcastHashJoin - : : : : :- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : +- ReusedSubquery - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : :- Project - : : : : +- BroadcastHashJoin - : : : : :- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- Project - : : : +- BroadcastHashJoin - : : : :- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- ReusedSubquery - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- HashAggregate - : : +- CometNativeColumnarToRow - : : +- CometColumnarExchange - : : +- HashAggregate - : : +- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- BroadcastHashJoin - : : : : :- Filter - : : : : : +- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : +- BroadcastExchange - : : : : +- Project - : : : : +- BroadcastHashJoin - : : : : :- CometNativeColumnarToRow - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : +- BroadcastExchange - : : : : +- BroadcastHashJoin - : : : : :- CometNativeColumnarToRow - : : : : : +- CometHashAggregate - : : : : : +- CometColumnarExchange - : : : : : +- HashAggregate - : : : : : +- Project - : : : : : +- BroadcastHashJoin - : : : : : :- Project - : : : : : : +- BroadcastHashJoin - : : : : : : :- Filter - : : : : : : : +- ColumnarToRow - : : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : : : +- SubqueryBroadcast - : : : : : : : +- BroadcastExchange - : : : : : : : +- CometNativeColumnarToRow - : : : : : : : +- CometProject - : : : : : : : +- CometFilter - : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : : : +- BroadcastExchange - : : : : : : +- BroadcastHashJoin - : : : : : : :- CometNativeColumnarToRow - : : : : : : : +- CometFilter - : : : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : : : +- BroadcastExchange - : : : : : : +- Project - : : : : : : +- BroadcastHashJoin - : : : : : : :- Project - : : : : : : : +- BroadcastHashJoin - : : : : : : : :- Filter - : : : : : : : : +- ColumnarToRow - : : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : : : : +- ReusedSubquery - : : : : : : : +- BroadcastExchange - : : : : : : : +- CometNativeColumnarToRow - : : : : : : : +- CometFilter - : : : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : : : +- BroadcastExchange - : : : : : : +- CometNativeColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : : +- BroadcastExchange - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : +- BroadcastExchange - : : : : +- Project - : : : : +- BroadcastHashJoin - : : : : :- Project - : : : : : +- BroadcastHashJoin - : : : : : :- Filter - : : : : : : +- ColumnarToRow - : : : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : : +- ReusedSubquery - : : : : : +- BroadcastExchange - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- BroadcastHashJoin - : : : :- CometNativeColumnarToRow - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : +- BroadcastExchange - : : : +- Project - : : : +- BroadcastHashJoin - : : : :- CometNativeColumnarToRow - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : +- BroadcastExchange - : : : +- BroadcastHashJoin - : : : :- CometNativeColumnarToRow - : : : : +- CometHashAggregate - : : : : +- CometColumnarExchange - : : : : +- HashAggregate - : : : : +- Project - : : : : +- BroadcastHashJoin - : : : : :- Project - : : : : : +- BroadcastHashJoin - : : : : : :- Filter - : : : : : : +- ColumnarToRow - : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometNativeColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : : +- BroadcastExchange - : : : : : +- BroadcastHashJoin - : : : : : :- CometNativeColumnarToRow - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : : +- BroadcastExchange - : : : : : +- Project - : : : : : +- BroadcastHashJoin - : : : : : :- Project - : : : : : : +- BroadcastHashJoin - : : : : : : :- Filter - : : : : : : : +- ColumnarToRow - : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : : : +- ReusedSubquery - : : : : : : +- BroadcastExchange - : : : : : : +- CometNativeColumnarToRow - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : : +- BroadcastExchange - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- Project - : : : +- BroadcastHashJoin - : : : :- Project - : : : : +- BroadcastHashJoin - : : : : :- Filter - : : : : : +- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : +- ReusedSubquery - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : :- Filter - : : : +- ReusedSubquery - : : +- HashAggregate - : : +- CometNativeColumnarToRow - : : +- CometColumnarExchange - : : +- HashAggregate - : : +- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- BroadcastHashJoin - : : : : :- Filter - : : : : : +- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : +- ReusedSubquery - : : : : +- BroadcastExchange - : : : : +- Project - : : : : +- BroadcastHashJoin - : : : : :- CometNativeColumnarToRow - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : +- BroadcastExchange - : : : : +- BroadcastHashJoin - : : : : :- CometNativeColumnarToRow - : : : : : +- CometHashAggregate - : : : : : +- CometColumnarExchange - : : : : : +- HashAggregate - : : : : : +- Project - : : : : : +- BroadcastHashJoin - : : : : : :- Project - : : : : : : +- BroadcastHashJoin - : : : : : : :- Filter - : : : : : : : +- ColumnarToRow - : : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : : : +- SubqueryBroadcast - : : : : : : : +- BroadcastExchange - : : : : : : : +- CometNativeColumnarToRow - : : : : : : : +- CometProject - : : : : : : : +- CometFilter - : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : : : +- BroadcastExchange - : : : : : : +- BroadcastHashJoin - : : : : : : :- CometNativeColumnarToRow - : : : : : : : +- CometFilter - : : : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : : : +- BroadcastExchange - : : : : : : +- Project - : : : : : : +- BroadcastHashJoin - : : : : : : :- Project - : : : : : : : +- BroadcastHashJoin - : : : : : : : :- Filter - : : : : : : : : +- ColumnarToRow - : : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : : : : +- ReusedSubquery - : : : : : : : +- BroadcastExchange - : : : : : : : +- CometNativeColumnarToRow - : : : : : : : +- CometFilter - : : : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : : : +- BroadcastExchange - : : : : : : +- CometNativeColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : : +- BroadcastExchange - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : +- BroadcastExchange - : : : : +- Project - : : : : +- BroadcastHashJoin - : : : : :- Project - : : : : : +- BroadcastHashJoin - : : : : : :- Filter - : : : : : : +- ColumnarToRow - : : : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : : +- ReusedSubquery - : : : : : +- BroadcastExchange - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- BroadcastHashJoin - : : : :- CometNativeColumnarToRow - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : +- BroadcastExchange - : : : +- Project - : : : +- BroadcastHashJoin - : : : :- CometNativeColumnarToRow - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : +- BroadcastExchange - : : : +- BroadcastHashJoin - : : : :- CometNativeColumnarToRow - : : : : +- CometHashAggregate - : : : : +- CometColumnarExchange - : : : : +- HashAggregate - : : : : +- Project - : : : : +- BroadcastHashJoin - : : : : :- Project - : : : : : +- BroadcastHashJoin - : : : : : :- Filter - : : : : : : +- ColumnarToRow - : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometNativeColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : : +- BroadcastExchange - : : : : : +- BroadcastHashJoin - : : : : : :- CometNativeColumnarToRow - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : : +- BroadcastExchange - : : : : : +- Project - : : : : : +- BroadcastHashJoin - : : : : : :- Project - : : : : : : +- BroadcastHashJoin - : : : : : : :- Filter - : : : : : : : +- ColumnarToRow - : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : : : +- ReusedSubquery - : : : : : : +- BroadcastExchange - : : : : : : +- CometNativeColumnarToRow - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : : +- BroadcastExchange - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- Project - : : : +- BroadcastHashJoin - : : : :- Project - : : : : +- BroadcastHashJoin - : : : : :- Filter - : : : : : +- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : +- ReusedSubquery - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- Filter - : : +- ReusedSubquery - : +- HashAggregate - : +- CometNativeColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- BroadcastHashJoin - : : : :- Filter - : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- ReusedSubquery - : : : +- BroadcastExchange - : : : +- Project - : : : +- BroadcastHashJoin - : : : :- CometNativeColumnarToRow - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : +- BroadcastExchange - : : : +- BroadcastHashJoin - : : : :- CometNativeColumnarToRow - : : : : +- CometHashAggregate - : : : : +- CometColumnarExchange - : : : : +- HashAggregate - : : : : +- Project - : : : : +- BroadcastHashJoin - : : : : :- Project - : : : : : +- BroadcastHashJoin - : : : : : :- Filter - : : : : : : +- ColumnarToRow - : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometNativeColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : : +- BroadcastExchange - : : : : : +- BroadcastHashJoin - : : : : : :- CometNativeColumnarToRow - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : : +- BroadcastExchange - : : : : : +- Project - : : : : : +- BroadcastHashJoin - : : : : : :- Project - : : : : : : +- BroadcastHashJoin - : : : : : : :- Filter - : : : : : : : +- ColumnarToRow - : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : : : +- ReusedSubquery - : : : : : : +- BroadcastExchange - : : : : : : +- CometNativeColumnarToRow - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : : +- BroadcastExchange - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- Project - : : : +- BroadcastHashJoin - : : : :- Project - : : : : +- BroadcastHashJoin - : : : : :- Filter - : : : : : +- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : +- ReusedSubquery - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- BroadcastHashJoin - : : :- CometNativeColumnarToRow - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.item - : : +- BroadcastExchange - : : +- Project - : : +- BroadcastHashJoin - : : :- CometNativeColumnarToRow - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.item - : : +- BroadcastExchange - : : +- BroadcastHashJoin - : : :- CometNativeColumnarToRow - : : : +- CometHashAggregate - : : : +- CometColumnarExchange - : : : +- HashAggregate - : : : +- Project - : : : +- BroadcastHashJoin - : : : :- Project - : : : : +- BroadcastHashJoin - : : : : :- Filter - : : : : : +- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : +- BroadcastExchange - : : : : +- BroadcastHashJoin - : : : : :- CometNativeColumnarToRow - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : +- BroadcastExchange - : : : : +- Project - : : : : +- BroadcastHashJoin - : : : : :- Project - : : : : : +- BroadcastHashJoin - : : : : : :- Filter - : : : : : : +- ColumnarToRow - : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : : +- ReusedSubquery - : : : : : +- BroadcastExchange - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- Filter - : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- ReusedSubquery - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.item - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- CometNativeColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.date_dim - +- HashAggregate - +- CometNativeColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- HashAggregate - +- CometNativeColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Union - :- Filter - : : +- Subquery - : : +- HashAggregate - : : +- CometNativeColumnarToRow - : : +- CometColumnarExchange - : : +- HashAggregate - : : +- Union - : : :- Project - : : : +- BroadcastHashJoin - : : : :- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- ReusedSubquery - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : :- Project - : : : +- BroadcastHashJoin - : : : :- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- Project - : : +- BroadcastHashJoin - : : :- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : +- ReusedSubquery - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- HashAggregate - : +- CometNativeColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- BroadcastHashJoin - : : : :- Filter - : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- Project - : : : +- BroadcastHashJoin - : : : :- CometNativeColumnarToRow - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : +- BroadcastExchange - : : : +- BroadcastHashJoin - : : : :- CometNativeColumnarToRow - : : : : +- CometHashAggregate - : : : : +- CometColumnarExchange - : : : : +- HashAggregate - : : : : +- Project - : : : : +- BroadcastHashJoin - : : : : :- Project - : : : : : +- BroadcastHashJoin - : : : : : :- Filter - : : : : : : +- ColumnarToRow - : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometNativeColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : : +- BroadcastExchange - : : : : : +- BroadcastHashJoin - : : : : : :- CometNativeColumnarToRow - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : : +- BroadcastExchange - : : : : : +- Project - : : : : : +- BroadcastHashJoin - : : : : : :- Project - : : : : : : +- BroadcastHashJoin - : : : : : : :- Filter - : : : : : : : +- ColumnarToRow - : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : : : +- ReusedSubquery - : : : : : : +- BroadcastExchange - : : : : : : +- CometNativeColumnarToRow - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : : +- BroadcastExchange - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- Project - : : : +- BroadcastHashJoin - : : : :- Project - : : : : +- BroadcastHashJoin - : : : : :- Filter - : : : : : +- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : +- ReusedSubquery - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- BroadcastHashJoin - : : :- CometNativeColumnarToRow - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.item - : : +- BroadcastExchange - : : +- Project - : : +- BroadcastHashJoin - : : :- CometNativeColumnarToRow - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.item - : : +- BroadcastExchange - : : +- BroadcastHashJoin - : : :- CometNativeColumnarToRow - : : : +- CometHashAggregate - : : : +- CometColumnarExchange - : : : +- HashAggregate - : : : +- Project - : : : +- BroadcastHashJoin - : : : :- Project - : : : : +- BroadcastHashJoin - : : : : :- Filter - : : : : : +- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : +- BroadcastExchange - : : : : +- BroadcastHashJoin - : : : : :- CometNativeColumnarToRow - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : +- BroadcastExchange - : : : : +- Project - : : : : +- BroadcastHashJoin - : : : : :- Project - : : : : : +- BroadcastHashJoin - : : : : : :- Filter - : : : : : : +- ColumnarToRow - : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : : +- ReusedSubquery - : : : : : +- BroadcastExchange - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- Filter - : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- ReusedSubquery - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.item - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- CometNativeColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.date_dim - :- Filter - : : +- ReusedSubquery - : +- HashAggregate - : +- CometNativeColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- BroadcastHashJoin - : : : :- Filter - : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- ReusedSubquery - : : : +- BroadcastExchange - : : : +- Project - : : : +- BroadcastHashJoin - : : : :- CometNativeColumnarToRow - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : +- BroadcastExchange - : : : +- BroadcastHashJoin - : : : :- CometNativeColumnarToRow - : : : : +- CometHashAggregate - : : : : +- CometColumnarExchange - : : : : +- HashAggregate - : : : : +- Project - : : : : +- BroadcastHashJoin - : : : : :- Project - : : : : : +- BroadcastHashJoin - : : : : : :- Filter - : : : : : : +- ColumnarToRow - : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometNativeColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : : +- BroadcastExchange - : : : : : +- BroadcastHashJoin - : : : : : :- CometNativeColumnarToRow - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : : +- BroadcastExchange - : : : : : +- Project - : : : : : +- BroadcastHashJoin - : : : : : :- Project - : : : : : : +- BroadcastHashJoin - : : : : : : :- Filter - : : : : : : : +- ColumnarToRow - : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : : : +- ReusedSubquery - : : : : : : +- BroadcastExchange - : : : : : : +- CometNativeColumnarToRow - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : : +- BroadcastExchange - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- Project - : : : +- BroadcastHashJoin - : : : :- Project - : : : : +- BroadcastHashJoin - : : : : :- Filter - : : : : : +- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : +- ReusedSubquery - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- BroadcastHashJoin - : : :- CometNativeColumnarToRow - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.item - : : +- BroadcastExchange - : : +- Project - : : +- BroadcastHashJoin - : : :- CometNativeColumnarToRow - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.item - : : +- BroadcastExchange - : : +- BroadcastHashJoin - : : :- CometNativeColumnarToRow - : : : +- CometHashAggregate - : : : +- CometColumnarExchange - : : : +- HashAggregate - : : : +- Project - : : : +- BroadcastHashJoin - : : : :- Project - : : : : +- BroadcastHashJoin - : : : : :- Filter - : : : : : +- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : +- BroadcastExchange - : : : : +- BroadcastHashJoin - : : : : :- CometNativeColumnarToRow - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : +- BroadcastExchange - : : : : +- Project - : : : : +- BroadcastHashJoin - : : : : :- Project - : : : : : +- BroadcastHashJoin - : : : : : :- Filter - : : : : : : +- ColumnarToRow - : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : : +- ReusedSubquery - : : : : : +- BroadcastExchange - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- Filter - : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- ReusedSubquery - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.item - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- CometNativeColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.date_dim - +- Filter - : +- ReusedSubquery - +- HashAggregate - +- CometNativeColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- BroadcastHashJoin - : : :- Filter - : : : +- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : +- ReusedSubquery - : : +- BroadcastExchange - : : +- Project - : : +- BroadcastHashJoin - : : :- CometNativeColumnarToRow - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.item - : : +- BroadcastExchange - : : +- BroadcastHashJoin - : : :- CometNativeColumnarToRow - : : : +- CometHashAggregate - : : : +- CometColumnarExchange - : : : +- HashAggregate - : : : +- Project - : : : +- BroadcastHashJoin - : : : :- Project - : : : : +- BroadcastHashJoin - : : : : :- Filter - : : : : : +- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : +- BroadcastExchange - : : : : +- BroadcastHashJoin - : : : : :- CometNativeColumnarToRow - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : +- BroadcastExchange - : : : : +- Project - : : : : +- BroadcastHashJoin - : : : : :- Project - : : : : : +- BroadcastHashJoin - : : : : : :- Filter - : : : : : : +- ColumnarToRow - : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : : +- ReusedSubquery - : : : : : +- BroadcastExchange - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- Filter - : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- ReusedSubquery - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.item - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- BroadcastHashJoin - : :- CometNativeColumnarToRow - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.item - : +- BroadcastExchange - : +- Project - : +- BroadcastHashJoin - : :- CometNativeColumnarToRow - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.item - : +- BroadcastExchange - : +- BroadcastHashJoin - : :- CometNativeColumnarToRow - : : +- CometHashAggregate - : : +- CometColumnarExchange - : : +- HashAggregate - : : +- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- Filter - : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- BroadcastHashJoin - : : : :- CometNativeColumnarToRow - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : +- BroadcastExchange - : : : +- Project - : : : +- BroadcastHashJoin - : : : :- Project - : : : : +- BroadcastHashJoin - : : : : :- Filter - : : : : : +- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : +- ReusedSubquery - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Filter - : : : +- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : +- ReusedSubquery - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.item - : +- BroadcastExchange - : +- CometNativeColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.date_dim - +- BroadcastExchange - +- CometNativeColumnarToRow - +- CometProject - +- CometFilter - +- CometNativeScan parquet spark_catalog.default.date_dim + :- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometUnion + : :- CometFilter + : : : +- Subquery + : : : +- CometNativeColumnarToRow + : : : +- CometHashAggregate + : : : +- CometExchange + : : : +- CometHashAggregate + : : : +- CometUnion + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometNativeScan parquet spark_catalog.default.store_sales + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometNativeColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometNativeColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometNativeScan parquet spark_catalog.default.web_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometNativeColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometHashAggregate + : : +- CometExchange + : : +- CometHashAggregate + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometBroadcastHashJoin + : : : : :- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometNativeColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : : +- CometBroadcastExchange + : : : : +- CometBroadcastHashJoin + : : : : :- CometHashAggregate + : : : : : +- CometExchange + : : : : : +- CometHashAggregate + : : : : : +- CometProject + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometProject + : : : : : : +- CometBroadcastHashJoin + : : : : : : :- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : : : : +- SubqueryBroadcast + : : : : : : : +- BroadcastExchange + : : : : : : : +- CometNativeColumnarToRow + : : : : : : : +- CometProject + : : : : : : : +- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : +- CometBroadcastExchange + : : : : : : +- CometBroadcastHashJoin + : : : : : : :- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : : : : +- CometBroadcastExchange + : : : : : : +- CometProject + : : : : : : +- CometBroadcastHashJoin + : : : : : : :- CometProject + : : : : : : : +- CometBroadcastHashJoin + : : : : : : : :- CometFilter + : : : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : : : : : +- SubqueryBroadcast + : : : : : : : : +- BroadcastExchange + : : : : : : : : +- CometNativeColumnarToRow + : : : : : : : : +- CometProject + : : : : : : : : +- CometFilter + : : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : : +- CometBroadcastExchange + : : : : : : : +- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : : : : +- CometBroadcastExchange + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- CometBroadcastExchange + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometProject + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.web_sales + : : : : : : +- SubqueryBroadcast + : : : : : : +- BroadcastExchange + : : : : : : +- CometNativeColumnarToRow + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- CometBroadcastExchange + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : +- CometBroadcastExchange + : : : +- CometBroadcastHashJoin + : : : :- CometHashAggregate + : : : : +- CometExchange + : : : : +- CometHashAggregate + : : : : +- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometProject + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : : : +- SubqueryBroadcast + : : : : : : +- BroadcastExchange + : : : : : : +- CometNativeColumnarToRow + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- CometBroadcastExchange + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : : : +- CometBroadcastExchange + : : : : : +- CometProject + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometProject + : : : : : : +- CometBroadcastHashJoin + : : : : : : :- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : : : : +- SubqueryBroadcast + : : : : : : : +- BroadcastExchange + : : : : : : : +- CometNativeColumnarToRow + : : : : : : : +- CometProject + : : : : : : : +- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : +- CometBroadcastExchange + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : : : +- CometBroadcastExchange + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.web_sales + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometNativeColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : :- CometFilter + : : : +- ReusedSubquery + : : +- CometHashAggregate + : : +- CometExchange + : : +- CometHashAggregate + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometBroadcastHashJoin + : : : : :- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometNativeColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : : +- CometBroadcastExchange + : : : : +- CometBroadcastHashJoin + : : : : :- CometHashAggregate + : : : : : +- CometExchange + : : : : : +- CometHashAggregate + : : : : : +- CometProject + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometProject + : : : : : : +- CometBroadcastHashJoin + : : : : : : :- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : : : : +- SubqueryBroadcast + : : : : : : : +- BroadcastExchange + : : : : : : : +- CometNativeColumnarToRow + : : : : : : : +- CometProject + : : : : : : : +- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : +- CometBroadcastExchange + : : : : : : +- CometBroadcastHashJoin + : : : : : : :- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : : : : +- CometBroadcastExchange + : : : : : : +- CometProject + : : : : : : +- CometBroadcastHashJoin + : : : : : : :- CometProject + : : : : : : : +- CometBroadcastHashJoin + : : : : : : : :- CometFilter + : : : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : : : : : +- SubqueryBroadcast + : : : : : : : : +- BroadcastExchange + : : : : : : : : +- CometNativeColumnarToRow + : : : : : : : : +- CometProject + : : : : : : : : +- CometFilter + : : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : : +- CometBroadcastExchange + : : : : : : : +- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : : : : +- CometBroadcastExchange + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- CometBroadcastExchange + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometProject + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.web_sales + : : : : : : +- SubqueryBroadcast + : : : : : : +- BroadcastExchange + : : : : : : +- CometNativeColumnarToRow + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- CometBroadcastExchange + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : +- CometBroadcastExchange + : : : +- CometBroadcastHashJoin + : : : :- CometHashAggregate + : : : : +- CometExchange + : : : : +- CometHashAggregate + : : : : +- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometProject + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : : : +- SubqueryBroadcast + : : : : : : +- BroadcastExchange + : : : : : : +- CometNativeColumnarToRow + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- CometBroadcastExchange + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : : : +- CometBroadcastExchange + : : : : : +- CometProject + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometProject + : : : : : : +- CometBroadcastHashJoin + : : : : : : :- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : : : : +- SubqueryBroadcast + : : : : : : : +- BroadcastExchange + : : : : : : : +- CometNativeColumnarToRow + : : : : : : : +- CometProject + : : : : : : : +- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : +- CometBroadcastExchange + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : : : +- CometBroadcastExchange + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.web_sales + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometNativeColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometFilter + : : +- ReusedSubquery + : +- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.web_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometNativeColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : +- CometBroadcastExchange + : : : +- CometBroadcastHashJoin + : : : :- CometHashAggregate + : : : : +- CometExchange + : : : : +- CometHashAggregate + : : : : +- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometProject + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : : : +- SubqueryBroadcast + : : : : : : +- BroadcastExchange + : : : : : : +- CometNativeColumnarToRow + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- CometBroadcastExchange + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : : : +- CometBroadcastExchange + : : : : : +- CometProject + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometProject + : : : : : : +- CometBroadcastHashJoin + : : : : : : :- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : : : : +- SubqueryBroadcast + : : : : : : : +- BroadcastExchange + : : : : : : : +- CometNativeColumnarToRow + : : : : : : : +- CometProject + : : : : : : : +- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : +- CometBroadcastExchange + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : : : +- CometBroadcastExchange + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.web_sales + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometNativeColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.item + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.item + : : +- CometBroadcastExchange + : : +- CometBroadcastHashJoin + : : :- CometHashAggregate + : : : +- CometExchange + : : : +- CometHashAggregate + : : : +- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometNativeColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometBroadcastHashJoin + : : : : :- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometProject + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : : : +- SubqueryBroadcast + : : : : : : +- BroadcastExchange + : : : : : : +- CometNativeColumnarToRow + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- CometBroadcastExchange + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.web_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometNativeColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.item + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.date_dim + :- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometUnion + : :- CometFilter + : : : +- Subquery + : : : +- CometNativeColumnarToRow + : : : +- CometHashAggregate + : : : +- CometExchange + : : : +- CometHashAggregate + : : : +- CometUnion + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometNativeScan parquet spark_catalog.default.store_sales + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometNativeColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometNativeColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometNativeScan parquet spark_catalog.default.web_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometNativeColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometHashAggregate + : : +- CometExchange + : : +- CometHashAggregate + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometBroadcastHashJoin + : : : : :- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometNativeColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : : +- CometBroadcastExchange + : : : : +- CometBroadcastHashJoin + : : : : :- CometHashAggregate + : : : : : +- CometExchange + : : : : : +- CometHashAggregate + : : : : : +- CometProject + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometProject + : : : : : : +- CometBroadcastHashJoin + : : : : : : :- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : : : : +- SubqueryBroadcast + : : : : : : : +- BroadcastExchange + : : : : : : : +- CometNativeColumnarToRow + : : : : : : : +- CometProject + : : : : : : : +- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : +- CometBroadcastExchange + : : : : : : +- CometBroadcastHashJoin + : : : : : : :- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : : : : +- CometBroadcastExchange + : : : : : : +- CometProject + : : : : : : +- CometBroadcastHashJoin + : : : : : : :- CometProject + : : : : : : : +- CometBroadcastHashJoin + : : : : : : : :- CometFilter + : : : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : : : : : +- SubqueryBroadcast + : : : : : : : : +- BroadcastExchange + : : : : : : : : +- CometNativeColumnarToRow + : : : : : : : : +- CometProject + : : : : : : : : +- CometFilter + : : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : : +- CometBroadcastExchange + : : : : : : : +- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : : : : +- CometBroadcastExchange + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- CometBroadcastExchange + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometProject + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.web_sales + : : : : : : +- SubqueryBroadcast + : : : : : : +- BroadcastExchange + : : : : : : +- CometNativeColumnarToRow + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- CometBroadcastExchange + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : +- CometBroadcastExchange + : : : +- CometBroadcastHashJoin + : : : :- CometHashAggregate + : : : : +- CometExchange + : : : : +- CometHashAggregate + : : : : +- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometProject + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : : : +- SubqueryBroadcast + : : : : : : +- BroadcastExchange + : : : : : : +- CometNativeColumnarToRow + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- CometBroadcastExchange + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : : : +- CometBroadcastExchange + : : : : : +- CometProject + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometProject + : : : : : : +- CometBroadcastHashJoin + : : : : : : :- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : : : : +- SubqueryBroadcast + : : : : : : : +- BroadcastExchange + : : : : : : : +- CometNativeColumnarToRow + : : : : : : : +- CometProject + : : : : : : : +- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : +- CometBroadcastExchange + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : : : +- CometBroadcastExchange + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.web_sales + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometNativeColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : :- CometFilter + : : : +- ReusedSubquery + : : +- CometHashAggregate + : : +- CometExchange + : : +- CometHashAggregate + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometBroadcastHashJoin + : : : : :- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometNativeColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : : +- CometBroadcastExchange + : : : : +- CometBroadcastHashJoin + : : : : :- CometHashAggregate + : : : : : +- CometExchange + : : : : : +- CometHashAggregate + : : : : : +- CometProject + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometProject + : : : : : : +- CometBroadcastHashJoin + : : : : : : :- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : : : : +- SubqueryBroadcast + : : : : : : : +- BroadcastExchange + : : : : : : : +- CometNativeColumnarToRow + : : : : : : : +- CometProject + : : : : : : : +- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : +- CometBroadcastExchange + : : : : : : +- CometBroadcastHashJoin + : : : : : : :- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : : : : +- CometBroadcastExchange + : : : : : : +- CometProject + : : : : : : +- CometBroadcastHashJoin + : : : : : : :- CometProject + : : : : : : : +- CometBroadcastHashJoin + : : : : : : : :- CometFilter + : : : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : : : : : +- SubqueryBroadcast + : : : : : : : : +- BroadcastExchange + : : : : : : : : +- CometNativeColumnarToRow + : : : : : : : : +- CometProject + : : : : : : : : +- CometFilter + : : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : : +- CometBroadcastExchange + : : : : : : : +- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : : : : +- CometBroadcastExchange + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- CometBroadcastExchange + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometProject + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.web_sales + : : : : : : +- SubqueryBroadcast + : : : : : : +- BroadcastExchange + : : : : : : +- CometNativeColumnarToRow + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- CometBroadcastExchange + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : +- CometBroadcastExchange + : : : +- CometBroadcastHashJoin + : : : :- CometHashAggregate + : : : : +- CometExchange + : : : : +- CometHashAggregate + : : : : +- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometProject + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : : : +- SubqueryBroadcast + : : : : : : +- BroadcastExchange + : : : : : : +- CometNativeColumnarToRow + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- CometBroadcastExchange + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : : : +- CometBroadcastExchange + : : : : : +- CometProject + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometProject + : : : : : : +- CometBroadcastHashJoin + : : : : : : :- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : : : : +- SubqueryBroadcast + : : : : : : : +- BroadcastExchange + : : : : : : : +- CometNativeColumnarToRow + : : : : : : : +- CometProject + : : : : : : : +- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : +- CometBroadcastExchange + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : : : +- CometBroadcastExchange + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.web_sales + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometNativeColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometFilter + : : +- ReusedSubquery + : +- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.web_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometNativeColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : +- CometBroadcastExchange + : : : +- CometBroadcastHashJoin + : : : :- CometHashAggregate + : : : : +- CometExchange + : : : : +- CometHashAggregate + : : : : +- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometProject + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : : : +- SubqueryBroadcast + : : : : : : +- BroadcastExchange + : : : : : : +- CometNativeColumnarToRow + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- CometBroadcastExchange + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : : : +- CometBroadcastExchange + : : : : : +- CometProject + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometProject + : : : : : : +- CometBroadcastHashJoin + : : : : : : :- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : : : : +- SubqueryBroadcast + : : : : : : : +- BroadcastExchange + : : : : : : : +- CometNativeColumnarToRow + : : : : : : : +- CometProject + : : : : : : : +- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : +- CometBroadcastExchange + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : : : +- CometBroadcastExchange + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.web_sales + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometNativeColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.item + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.item + : : +- CometBroadcastExchange + : : +- CometBroadcastHashJoin + : : :- CometHashAggregate + : : : +- CometExchange + : : : +- CometHashAggregate + : : : +- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometNativeColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometBroadcastHashJoin + : : : : :- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometProject + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : : : +- SubqueryBroadcast + : : : : : : +- BroadcastExchange + : : : : : : +- CometNativeColumnarToRow + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- CometBroadcastExchange + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.web_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometNativeColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.item + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.date_dim + +- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometUnion + :- CometFilter + : : +- Subquery + : : +- CometNativeColumnarToRow + : : +- CometHashAggregate + : : +- CometExchange + : : +- CometHashAggregate + : : +- CometUnion + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometNativeScan parquet spark_catalog.default.store_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometNativeColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometNativeColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometNativeScan parquet spark_catalog.default.web_sales + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometNativeColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometNativeColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : +- CometBroadcastExchange + : : : +- CometBroadcastHashJoin + : : : :- CometHashAggregate + : : : : +- CometExchange + : : : : +- CometHashAggregate + : : : : +- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometProject + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : : : +- SubqueryBroadcast + : : : : : : +- BroadcastExchange + : : : : : : +- CometNativeColumnarToRow + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- CometBroadcastExchange + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : : : +- CometBroadcastExchange + : : : : : +- CometProject + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometProject + : : : : : : +- CometBroadcastHashJoin + : : : : : : :- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : : : : +- SubqueryBroadcast + : : : : : : : +- BroadcastExchange + : : : : : : : +- CometNativeColumnarToRow + : : : : : : : +- CometProject + : : : : : : : +- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : +- CometBroadcastExchange + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : : : +- CometBroadcastExchange + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.web_sales + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometNativeColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.item + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.item + : : +- CometBroadcastExchange + : : +- CometBroadcastHashJoin + : : :- CometHashAggregate + : : : +- CometExchange + : : : +- CometHashAggregate + : : : +- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometNativeColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometBroadcastHashJoin + : : : : :- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometProject + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : : : +- SubqueryBroadcast + : : : : : : +- BroadcastExchange + : : : : : : +- CometNativeColumnarToRow + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- CometBroadcastExchange + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.web_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometNativeColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.item + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.date_dim + :- CometFilter + : : +- ReusedSubquery + : +- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometNativeColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : +- CometBroadcastExchange + : : : +- CometBroadcastHashJoin + : : : :- CometHashAggregate + : : : : +- CometExchange + : : : : +- CometHashAggregate + : : : : +- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometProject + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : : : +- SubqueryBroadcast + : : : : : : +- BroadcastExchange + : : : : : : +- CometNativeColumnarToRow + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- CometBroadcastExchange + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : : : +- CometBroadcastExchange + : : : : : +- CometProject + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometProject + : : : : : : +- CometBroadcastHashJoin + : : : : : : :- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : : : : +- SubqueryBroadcast + : : : : : : : +- BroadcastExchange + : : : : : : : +- CometNativeColumnarToRow + : : : : : : : +- CometProject + : : : : : : : +- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : +- CometBroadcastExchange + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : : : +- CometBroadcastExchange + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.web_sales + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometNativeColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.item + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.item + : : +- CometBroadcastExchange + : : +- CometBroadcastHashJoin + : : :- CometHashAggregate + : : : +- CometExchange + : : : +- CometHashAggregate + : : : +- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometNativeColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometBroadcastHashJoin + : : : : :- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometProject + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : : : +- SubqueryBroadcast + : : : : : : +- BroadcastExchange + : : : : : : +- CometNativeColumnarToRow + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- CometBroadcastExchange + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.web_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometNativeColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.item + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.date_dim + +- CometFilter + : +- ReusedSubquery + +- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.web_sales + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometNativeColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.item + : : +- CometBroadcastExchange + : : +- CometBroadcastHashJoin + : : :- CometHashAggregate + : : : +- CometExchange + : : : +- CometHashAggregate + : : : +- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometNativeColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometBroadcastHashJoin + : : : : :- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometProject + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : : : +- SubqueryBroadcast + : : : : : : +- BroadcastExchange + : : : : : : +- CometNativeColumnarToRow + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- CometBroadcastExchange + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.web_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometNativeColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.item + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometBroadcastHashJoin + : :- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.item + : +- CometBroadcastExchange + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.item + : +- CometBroadcastExchange + : +- CometBroadcastHashJoin + : :- CometHashAggregate + : : +- CometExchange + : : +- CometHashAggregate + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometNativeColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometNativeColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.web_sales + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometNativeColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.item + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.date_dim + +- CometBroadcastExchange + +- CometProject + +- CometFilter + +- CometNativeScan parquet spark_catalog.default.date_dim -Comet accelerated 842 out of 2302 eligible operators (36%). Final plan contains 475 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 2367 out of 2622 eligible operators (90%). Final plan contains 126 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q18a.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q18a.native_datafusion/extended.txt index 8ad4e890ec..82bacea2fb 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q18a.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q18a.native_datafusion/extended.txt @@ -1,253 +1,234 @@ -TakeOrderedAndProject -+- Union - :- HashAggregate - : +- CometNativeColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- Project - : : : : +- BroadcastHashJoin - : : : : :- Project - : : : : : +- BroadcastHashJoin - : : : : : :- Project - : : : : : : +- BroadcastHashJoin - : : : : : : :- Filter - : : : : : : : +- ColumnarToRow - : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : : : +- SubqueryBroadcast - : : : : : : : +- BroadcastExchange - : : : : : : : +- CometNativeColumnarToRow - : : : : : : : +- CometProject - : : : : : : : +- CometFilter - : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : : : +- BroadcastExchange - : : : : : : +- CometNativeColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.customer_demographics - : : : : : +- BroadcastExchange - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.customer - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.customer_demographics - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.customer_address - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- CometNativeColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.item - :- HashAggregate - : +- CometNativeColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- Project - : : : : +- BroadcastHashJoin - : : : : :- Project - : : : : : +- BroadcastHashJoin - : : : : : :- Project - : : : : : : +- BroadcastHashJoin - : : : : : : :- Filter - : : : : : : : +- ColumnarToRow - : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : : : +- ReusedSubquery - : : : : : : +- BroadcastExchange - : : : : : : +- CometNativeColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.customer_demographics - : : : : : +- BroadcastExchange - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.customer - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.customer_demographics - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.customer_address - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- CometNativeColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.item - :- HashAggregate - : +- CometNativeColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- Project - : : : : +- BroadcastHashJoin - : : : : :- Project - : : : : : +- BroadcastHashJoin - : : : : : :- Project - : : : : : : +- BroadcastHashJoin - : : : : : : :- Filter - : : : : : : : +- ColumnarToRow - : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : : : +- ReusedSubquery - : : : : : : +- BroadcastExchange - : : : : : : +- CometNativeColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.customer_demographics - : : : : : +- BroadcastExchange - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.customer - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.customer_demographics - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.customer_address - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- CometNativeColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.item - :- HashAggregate - : +- CometNativeColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- Project - : : : : +- BroadcastHashJoin - : : : : :- Project - : : : : : +- BroadcastHashJoin - : : : : : :- Project - : : : : : : +- BroadcastHashJoin - : : : : : : :- Filter - : : : : : : : +- ColumnarToRow - : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : : : +- ReusedSubquery - : : : : : : +- BroadcastExchange - : : : : : : +- CometNativeColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.customer_demographics - : : : : : +- BroadcastExchange - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.customer - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.customer_demographics - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.customer_address - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- CometNativeColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.item - +- HashAggregate - +- CometNativeColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- Project - : : : : +- BroadcastHashJoin - : : : : :- Project - : : : : : +- BroadcastHashJoin - : : : : : :- Filter - : : : : : : +- ColumnarToRow - : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : : +- ReusedSubquery - : : : : : +- BroadcastExchange - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.customer_demographics - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.customer - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.customer_demographics - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.customer_address - : +- BroadcastExchange - : +- CometNativeColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.date_dim - +- BroadcastExchange - +- CometNativeColumnarToRow - +- CometFilter - +- CometNativeScan parquet spark_catalog.default.item +CometNativeColumnarToRow ++- CometTakeOrderedAndProject + +- CometUnion + :- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometProject + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometProject + : : : : : : +- CometBroadcastHashJoin + : : : : : : :- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : : : : +- SubqueryBroadcast + : : : : : : : +- BroadcastExchange + : : : : : : : +- CometNativeColumnarToRow + : : : : : : : +- CometProject + : : : : : : : +- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : +- CometBroadcastExchange + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.customer_demographics + : : : : : +- CometBroadcastExchange + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.customer + : : : : +- CometBroadcastExchange + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.customer_demographics + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.customer_address + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.item + :- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometProject + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometProject + : : : : : : +- CometBroadcastHashJoin + : : : : : : :- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : : : : +- SubqueryBroadcast + : : : : : : : +- BroadcastExchange + : : : : : : : +- CometNativeColumnarToRow + : : : : : : : +- CometProject + : : : : : : : +- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : +- CometBroadcastExchange + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.customer_demographics + : : : : : +- CometBroadcastExchange + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.customer + : : : : +- CometBroadcastExchange + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.customer_demographics + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.customer_address + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.item + :- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometProject + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometProject + : : : : : : +- CometBroadcastHashJoin + : : : : : : :- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : : : : +- SubqueryBroadcast + : : : : : : : +- BroadcastExchange + : : : : : : : +- CometNativeColumnarToRow + : : : : : : : +- CometProject + : : : : : : : +- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : +- CometBroadcastExchange + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.customer_demographics + : : : : : +- CometBroadcastExchange + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.customer + : : : : +- CometBroadcastExchange + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.customer_demographics + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.customer_address + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.item + :- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometProject + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometProject + : : : : : : +- CometBroadcastHashJoin + : : : : : : :- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : : : : +- SubqueryBroadcast + : : : : : : : +- BroadcastExchange + : : : : : : : +- CometNativeColumnarToRow + : : : : : : : +- CometProject + : : : : : : : +- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : +- CometBroadcastExchange + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.customer_demographics + : : : : : +- CometBroadcastExchange + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.customer + : : : : +- CometBroadcastExchange + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.customer_demographics + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.customer_address + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.item + +- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometProject + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : : : +- SubqueryBroadcast + : : : : : : +- BroadcastExchange + : : : : : : +- CometNativeColumnarToRow + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- CometBroadcastExchange + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.customer_demographics + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.customer + : : : +- CometBroadcastExchange + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.customer_demographics + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.customer_address + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.date_dim + +- CometBroadcastExchange + +- CometFilter + +- CometNativeScan parquet spark_catalog.default.item -Comet accelerated 92 out of 210 eligible operators (43%). Final plan contains 41 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 216 out of 226 eligible operators (95%). Final plan contains 6 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q20.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q20.native_datafusion/extended.txt index 6634edcbed..c4f54f4289 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q20.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q20.native_datafusion/extended.txt @@ -3,33 +3,29 @@ TakeOrderedAndProject +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] +- CometNativeColumnarToRow +- CometSort - +- CometColumnarExchange - +- HashAggregate - +- CometNativeColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- Filter - : : +- ColumnarToRow - : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- CometNativeColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.item - +- BroadcastExchange - +- CometNativeColumnarToRow - +- CometProject - +- CometFilter - +- CometNativeScan parquet spark_catalog.default.date_dim + +- CometExchange + +- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : +- SubqueryBroadcast + : : +- BroadcastExchange + : : +- CometNativeColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.item + +- CometBroadcastExchange + +- CometProject + +- CometFilter + +- CometNativeScan parquet spark_catalog.default.date_dim -Comet accelerated 12 out of 27 eligible operators (44%). Final plan contains 6 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 22 out of 27 eligible operators (81%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q22.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q22.native_datafusion/extended.txt index 108e3c13f9..a8588fd031 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q22.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q22.native_datafusion/extended.txt @@ -5,27 +5,25 @@ TakeOrderedAndProject +- HashAggregate +- Expand +- Project - +- BroadcastNestedLoopJoin - :- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Filter - : : : +- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.inventory [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- CometNativeColumnarToRow + +- BroadcastNestedLoopJoin [COMET: BroadcastNestedLoopJoin is not supported] + :- CometNativeColumnarToRow + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.inventory + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometNativeColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange : +- CometProject : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.item @@ -33,4 +31,4 @@ TakeOrderedAndProject +- CometNativeColumnarToRow +- CometNativeScan parquet spark_catalog.default.warehouse -Comet accelerated 11 out of 28 eligible operators (39%). Final plan contains 6 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 19 out of 28 eligible operators (67%). Final plan contains 4 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q22a.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q22a.native_datafusion/extended.txt index eaa5438d91..5bd5daea84 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q22a.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q22a.native_datafusion/extended.txt @@ -1,187 +1,159 @@ -TakeOrderedAndProject -+- Union - :- HashAggregate - : +- HashAggregate - : +- HashAggregate - : +- CometNativeColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- Filter - : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.inventory [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.item - : +- BroadcastExchange - : +- CometNativeColumnarToRow - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.warehouse - :- HashAggregate - : +- CometNativeColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- HashAggregate - : +- CometNativeColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- Filter - : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.inventory [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.item - : +- BroadcastExchange - : +- CometNativeColumnarToRow - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.warehouse - :- HashAggregate - : +- CometNativeColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- HashAggregate - : +- CometNativeColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- Filter - : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.inventory [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.item - : +- BroadcastExchange - : +- CometNativeColumnarToRow - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.warehouse - :- HashAggregate - : +- CometNativeColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- HashAggregate - : +- CometNativeColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- Filter - : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.inventory [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.item - : +- BroadcastExchange - : +- CometNativeColumnarToRow - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.warehouse - +- HashAggregate - +- CometNativeColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- HashAggregate - +- CometNativeColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Filter - : : : +- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.inventory [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- CometNativeColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.item - +- BroadcastExchange - +- CometNativeColumnarToRow - +- CometFilter - +- CometNativeScan parquet spark_catalog.default.warehouse +CometNativeColumnarToRow ++- CometTakeOrderedAndProject + +- CometUnion + :- CometHashAggregate + : +- CometHashAggregate + : +- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.inventory + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometNativeColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.item + : +- CometBroadcastExchange + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.warehouse + :- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.inventory + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometNativeColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.item + : +- CometBroadcastExchange + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.warehouse + :- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.inventory + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometNativeColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.item + : +- CometBroadcastExchange + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.warehouse + :- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.inventory + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometNativeColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.item + : +- CometBroadcastExchange + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.warehouse + +- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.inventory + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometNativeColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.item + +- CometBroadcastExchange + +- CometFilter + +- CometNativeScan parquet spark_catalog.default.warehouse -Comet accelerated 64 out of 151 eligible operators (42%). Final plan contains 34 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 141 out of 151 eligible operators (93%). Final plan contains 6 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q27a.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q27a.native_datafusion/extended.txt index 4eec8b0953..5dc06ab86f 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q27a.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q27a.native_datafusion/extended.txt @@ -1,116 +1,109 @@ -TakeOrderedAndProject -+- Union - :- HashAggregate - : +- CometNativeColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- Project - : : : : +- BroadcastHashJoin - : : : : :- Filter - : : : : : +- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.customer_demographics - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.store - : +- BroadcastExchange - : +- CometNativeColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.item - :- HashAggregate - : +- CometNativeColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- Project - : : : : +- BroadcastHashJoin - : : : : :- Filter - : : : : : +- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : +- ReusedSubquery - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.customer_demographics - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.store - : +- BroadcastExchange - : +- CometNativeColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.item - +- HashAggregate - +- CometNativeColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- Filter - : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- ReusedSubquery - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.customer_demographics - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- CometNativeColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.store - +- BroadcastExchange - +- CometNativeColumnarToRow - +- CometFilter - +- CometNativeScan parquet spark_catalog.default.item +CometNativeColumnarToRow ++- CometTakeOrderedAndProject + +- CometUnion + :- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometNativeColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.customer_demographics + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.store + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.item + :- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometNativeColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.customer_demographics + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.store + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.item + +- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometNativeColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.customer_demographics + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.store + +- CometBroadcastExchange + +- CometFilter + +- CometNativeScan parquet spark_catalog.default.item -Comet accelerated 41 out of 95 eligible operators (43%). Final plan contains 19 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 97 out of 103 eligible operators (94%). Final plan contains 4 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q34.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q34.native_datafusion/extended.txt index 40230582bb..1b61be2937 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q34.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q34.native_datafusion/extended.txt @@ -1,47 +1,41 @@ CometNativeColumnarToRow +- CometSort - +- CometColumnarExchange - +- Project - +- BroadcastHashJoin - :- Filter - : +- HashAggregate - : +- CometNativeColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- Filter - : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.store - : +- BroadcastExchange - : +- CometNativeColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.household_demographics - +- BroadcastExchange - +- CometNativeColumnarToRow - +- CometProject - +- CometFilter - +- CometNativeScan parquet spark_catalog.default.customer + +- CometExchange + +- CometProject + +- CometBroadcastHashJoin + :- CometFilter + : +- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometNativeColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.store + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.household_demographics + +- CometBroadcastExchange + +- CometProject + +- CometFilter + +- CometNativeScan parquet spark_catalog.default.customer -Comet accelerated 18 out of 37 eligible operators (48%). Final plan contains 8 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 35 out of 37 eligible operators (94%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q35.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q35.native_datafusion/extended.txt index 40a8ef4a6e..2785c3d0b1 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q35.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q35.native_datafusion/extended.txt @@ -10,46 +10,52 @@ TakeOrderedAndProject : :- Project : : +- Filter : : +- BroadcastHashJoin - : : :- BroadcastHashJoin - : : : :- BroadcastHashJoin - : : : : :- CometNativeColumnarToRow - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.customer - : : : : +- BroadcastExchange - : : : : +- Project - : : : : +- BroadcastHashJoin - : : : : :- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow + : : :- BroadcastHashJoin [COMET: Unsupported join type ExistenceJoin(exists#1)] + : : : :- CometNativeColumnarToRow + : : : : +- CometBroadcastHashJoin + : : : : :- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.customer + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometNativeScan parquet spark_catalog.default.store_sales + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometNativeColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange : : : : +- CometProject : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- BroadcastExchange - : : : +- Project - : : : +- BroadcastHashJoin - : : : :- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- ReusedSubquery - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow + : : : +- CometNativeColumnarToRow + : : : +- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometNativeScan parquet spark_catalog.default.web_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometNativeColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- BroadcastExchange - : : +- Project - : : +- BroadcastHashJoin - : : :- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : +- ReusedSubquery - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow + : : +- CometNativeColumnarToRow + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometNativeColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange : : +- CometProject : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -64,4 +70,4 @@ TakeOrderedAndProject +- CometFilter +- CometNativeScan parquet spark_catalog.default.customer_demographics -Comet accelerated 21 out of 54 eligible operators (38%). Final plan contains 11 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 41 out of 62 eligible operators (66%). Final plan contains 9 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q35a.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q35a.native_datafusion/extended.txt index 9091536e96..cba125f412 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q35a.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q35a.native_datafusion/extended.txt @@ -1,65 +1,66 @@ -TakeOrderedAndProject -+- HashAggregate - +- CometNativeColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- BroadcastHashJoin - : : : :- CometNativeColumnarToRow - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.customer - : : : +- BroadcastExchange - : : : +- Project - : : : +- BroadcastHashJoin - : : : :- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- Union - : : :- Project - : : : +- BroadcastHashJoin - : : : :- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- ReusedSubquery - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- Project - : : +- BroadcastHashJoin - : : :- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : +- ReusedSubquery - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- CometNativeColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.customer_address - +- BroadcastExchange - +- CometNativeColumnarToRow - +- CometProject - +- CometFilter - +- CometNativeScan parquet spark_catalog.default.customer_demographics +CometNativeColumnarToRow ++- CometTakeOrderedAndProject + +- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.customer + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometNativeScan parquet spark_catalog.default.store_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometNativeColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometUnion + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometNativeScan parquet spark_catalog.default.web_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometNativeColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometNativeColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.customer_address + +- CometBroadcastExchange + +- CometProject + +- CometFilter + +- CometNativeScan parquet spark_catalog.default.customer_demographics -Comet accelerated 21 out of 52 eligible operators (40%). Final plan contains 11 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 54 out of 60 eligible operators (90%). Final plan contains 4 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q36a.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q36a.native_datafusion/extended.txt index 1fc86dc5a8..0a69bca968 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q36a.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q36a.native_datafusion/extended.txt @@ -5,118 +5,101 @@ TakeOrderedAndProject +- CometSort +- CometExchange +- CometHashAggregate - +- CometColumnarExchange - +- HashAggregate - +- Union - :- HashAggregate - : +- CometNativeColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- Filter - : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.item - : +- BroadcastExchange - : +- CometNativeColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.store - :- HashAggregate - : +- CometNativeColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- HashAggregate - : +- CometNativeColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- Filter - : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.item - : +- BroadcastExchange - : +- CometNativeColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.store - +- HashAggregate - +- CometNativeColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- HashAggregate - +- CometNativeColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Filter - : : : +- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- CometNativeColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.item - +- BroadcastExchange - +- CometNativeColumnarToRow - +- CometProject - +- CometFilter - +- CometNativeScan parquet spark_catalog.default.store + +- CometExchange + +- CometHashAggregate + +- CometUnion + :- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometNativeColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.item + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.store + :- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometNativeColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.item + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.store + +- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometNativeColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.item + +- CometBroadcastExchange + +- CometProject + +- CometFilter + +- CometNativeScan parquet spark_catalog.default.store -Comet accelerated 45 out of 99 eligible operators (45%). Final plan contains 21 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 90 out of 99 eligible operators (90%). Final plan contains 4 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q47.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q47.native_datafusion/extended.txt index ab44bbef5c..0886059452 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q47.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q47.native_datafusion/extended.txt @@ -10,111 +10,96 @@ TakeOrderedAndProject : : +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] : : +- CometNativeColumnarToRow : : +- CometSort - : : +- CometColumnarExchange - : : +- HashAggregate - : : +- CometNativeColumnarToRow - : : +- CometColumnarExchange - : : +- HashAggregate - : : +- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- Project - : : : : +- BroadcastHashJoin - : : : : :- CometNativeColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : +- BroadcastExchange - : : : : +- Filter - : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.store + : : +- CometExchange + : : +- CometHashAggregate + : : +- CometExchange + : : +- CometHashAggregate + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : : +- CometBroadcastExchange + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometNativeColumnarToRow + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.store : +- BroadcastExchange : +- Project : +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] : +- CometNativeColumnarToRow : +- CometSort - : +- CometColumnarExchange - : +- HashAggregate - : +- CometNativeColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : +- BroadcastExchange - : : : +- Filter - : : : +- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- CometNativeColumnarToRow - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.store + : +- CometExchange + : +- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : +- CometBroadcastExchange + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometNativeColumnarToRow + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.store +- BroadcastExchange +- Project +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] +- CometNativeColumnarToRow +- CometSort - +- CometColumnarExchange - +- HashAggregate - +- CometNativeColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.item - : : +- BroadcastExchange - : : +- Filter - : : +- ColumnarToRow - : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- CometNativeColumnarToRow - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.date_dim - +- BroadcastExchange - +- CometNativeColumnarToRow - +- CometFilter - +- CometNativeScan parquet spark_catalog.default.store + +- CometExchange + +- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.item + : : +- CometBroadcastExchange + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : +- SubqueryBroadcast + : : +- BroadcastExchange + : : +- CometNativeColumnarToRow + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.date_dim + +- CometBroadcastExchange + +- CometFilter + +- CometNativeScan parquet spark_catalog.default.store -Comet accelerated 36 out of 97 eligible operators (37%). Final plan contains 21 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 75 out of 97 eligible operators (77%). Final plan contains 6 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q49.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q49.native_datafusion/extended.txt index 0090739d55..6d7dadcde9 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q49.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q49.native_datafusion/extended.txt @@ -11,35 +11,31 @@ CometNativeColumnarToRow : +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] : +- CometNativeColumnarToRow : +- CometSort - : +- CometColumnarExchange - : +- HashAggregate - : +- CometNativeColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- BroadcastExchange - : : : +- Project - : : : +- Filter - : : : +- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- CometNativeColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.web_returns - : +- BroadcastExchange - : +- CometNativeColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometExchange + : +- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.web_sales + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometNativeColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.web_returns + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.date_dim :- Project : +- Filter : +- Window @@ -47,30 +43,31 @@ CometNativeColumnarToRow : +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] : +- CometNativeColumnarToRow : +- CometSort - : +- CometColumnarExchange - : +- HashAggregate - : +- CometNativeColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- BroadcastExchange - : : : +- Project - : : : +- Filter - : : : +- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : +- ReusedSubquery - : : +- CometNativeColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.catalog_returns - : +- BroadcastExchange - : +- CometNativeColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometExchange + : +- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometNativeColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.catalog_returns + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.date_dim +- Project +- Filter +- Window @@ -78,29 +75,30 @@ CometNativeColumnarToRow +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] +- CometNativeColumnarToRow +- CometSort - +- CometColumnarExchange - +- HashAggregate - +- CometNativeColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- BroadcastExchange - : : +- Project - : : +- Filter - : : +- ColumnarToRow - : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : +- ReusedSubquery - : +- CometNativeColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.store_returns - +- BroadcastExchange - +- CometNativeColumnarToRow - +- CometProject - +- CometFilter - +- CometNativeScan parquet spark_catalog.default.date_dim + +- CometExchange + +- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : +- SubqueryBroadcast + : : +- BroadcastExchange + : : +- CometNativeColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.store_returns + +- CometBroadcastExchange + +- CometProject + +- CometFilter + +- CometNativeScan parquet spark_catalog.default.date_dim -Comet accelerated 33 out of 87 eligible operators (37%). Final plan contains 17 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 72 out of 95 eligible operators (75%). Final plan contains 7 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q51a.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q51a.native_datafusion/extended.txt index 92ee33eda6..d9b9186c1e 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q51a.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q51a.native_datafusion/extended.txt @@ -23,53 +23,47 @@ TakeOrderedAndProject : : : +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] : : : +- CometNativeColumnarToRow : : : +- CometSort - : : : +- CometColumnarExchange - : : : +- HashAggregate - : : : +- CometNativeColumnarToRow - : : : +- CometColumnarExchange - : : : +- HashAggregate - : : : +- Project - : : : +- BroadcastHashJoin - : : : :- Filter - : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometExchange + : : : +- CometHashAggregate + : : : +- CometExchange + : : : +- CometHashAggregate + : : : +- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.web_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometNativeColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- BroadcastExchange : : +- Project : : +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] : : +- CometNativeColumnarToRow : : +- CometSort - : : +- CometColumnarExchange - : : +- HashAggregate - : : +- CometNativeColumnarToRow - : : +- CometColumnarExchange - : : +- HashAggregate - : : +- Project - : : +- BroadcastHashJoin - : : :- Filter - : : : +- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometExchange + : : +- CometHashAggregate + : : +- CometExchange + : : +- CometHashAggregate + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.web_sales + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometNativeColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim : +- CometSort : +- CometColumnarExchange : +- HashAggregate @@ -82,43 +76,47 @@ TakeOrderedAndProject : : +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] : : +- CometNativeColumnarToRow : : +- CometSort - : : +- CometColumnarExchange - : : +- HashAggregate - : : +- CometNativeColumnarToRow - : : +- CometColumnarExchange - : : +- HashAggregate - : : +- Project - : : +- BroadcastHashJoin - : : :- Filter - : : : +- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : +- ReusedSubquery - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometExchange + : : +- CometHashAggregate + : : +- CometExchange + : : +- CometHashAggregate + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometNativeColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim : +- BroadcastExchange : +- Project : +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] : +- CometNativeColumnarToRow : +- CometSort - : +- CometColumnarExchange - : +- HashAggregate - : +- CometNativeColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- Filter - : : +- ColumnarToRow - : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : +- ReusedSubquery - : +- BroadcastExchange - : +- CometNativeColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometExchange + : +- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : +- SubqueryBroadcast + : : +- BroadcastExchange + : : +- CometNativeColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.date_dim +- BroadcastExchange +- Project +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] @@ -140,53 +138,47 @@ TakeOrderedAndProject : : +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] : : +- CometNativeColumnarToRow : : +- CometSort - : : +- CometColumnarExchange - : : +- HashAggregate - : : +- CometNativeColumnarToRow - : : +- CometColumnarExchange - : : +- HashAggregate - : : +- Project - : : +- BroadcastHashJoin - : : :- Filter - : : : +- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometExchange + : : +- CometHashAggregate + : : +- CometExchange + : : +- CometHashAggregate + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.web_sales + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometNativeColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim : +- BroadcastExchange : +- Project : +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] : +- CometNativeColumnarToRow : +- CometSort - : +- CometColumnarExchange - : +- HashAggregate - : +- CometNativeColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- Filter - : : +- ColumnarToRow - : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- CometNativeColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometExchange + : +- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.web_sales + : : +- SubqueryBroadcast + : : +- BroadcastExchange + : : +- CometNativeColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.date_dim +- CometSort +- CometColumnarExchange +- HashAggregate @@ -199,42 +191,46 @@ TakeOrderedAndProject : +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] : +- CometNativeColumnarToRow : +- CometSort - : +- CometColumnarExchange - : +- HashAggregate - : +- CometNativeColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- Filter - : : +- ColumnarToRow - : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : +- ReusedSubquery - : +- BroadcastExchange - : +- CometNativeColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometExchange + : +- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : +- SubqueryBroadcast + : : +- BroadcastExchange + : : +- CometNativeColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.date_dim +- BroadcastExchange +- Project +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] +- CometNativeColumnarToRow +- CometSort - +- CometColumnarExchange - +- HashAggregate - +- CometNativeColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Project - +- BroadcastHashJoin - :- Filter - : +- ColumnarToRow - : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : +- ReusedSubquery - +- BroadcastExchange - +- CometNativeColumnarToRow - +- CometProject - +- CometFilter - +- CometNativeScan parquet spark_catalog.default.date_dim + +- CometExchange + +- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometProject + +- CometBroadcastHashJoin + :- CometFilter + : +- CometNativeScan parquet spark_catalog.default.store_sales + : +- SubqueryBroadcast + : +- BroadcastExchange + : +- CometNativeColumnarToRow + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.date_dim + +- CometBroadcastExchange + +- CometProject + +- CometFilter + +- CometNativeScan parquet spark_catalog.default.date_dim -Comet accelerated 82 out of 196 eligible operators (41%). Final plan contains 42 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 150 out of 212 eligible operators (70%). Final plan contains 22 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q57.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q57.native_datafusion/extended.txt index 66c510aaa0..024bfeeeea 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q57.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q57.native_datafusion/extended.txt @@ -10,111 +10,96 @@ TakeOrderedAndProject : : +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] : : +- CometNativeColumnarToRow : : +- CometSort - : : +- CometColumnarExchange - : : +- HashAggregate - : : +- CometNativeColumnarToRow - : : +- CometColumnarExchange - : : +- HashAggregate - : : +- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- Project - : : : : +- BroadcastHashJoin - : : : : :- CometNativeColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : +- BroadcastExchange - : : : : +- Filter - : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.call_center + : : +- CometExchange + : : +- CometHashAggregate + : : +- CometExchange + : : +- CometHashAggregate + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : : +- CometBroadcastExchange + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometNativeColumnarToRow + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.call_center : +- BroadcastExchange : +- Project : +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] : +- CometNativeColumnarToRow : +- CometSort - : +- CometColumnarExchange - : +- HashAggregate - : +- CometNativeColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : +- BroadcastExchange - : : : +- Filter - : : : +- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- CometNativeColumnarToRow - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.call_center + : +- CometExchange + : +- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : +- CometBroadcastExchange + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometNativeColumnarToRow + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.call_center +- BroadcastExchange +- Project +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] +- CometNativeColumnarToRow +- CometSort - +- CometColumnarExchange - +- HashAggregate - +- CometNativeColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.item - : : +- BroadcastExchange - : : +- Filter - : : +- ColumnarToRow - : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- CometNativeColumnarToRow - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.date_dim - +- BroadcastExchange - +- CometNativeColumnarToRow - +- CometFilter - +- CometNativeScan parquet spark_catalog.default.call_center + +- CometExchange + +- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.item + : : +- CometBroadcastExchange + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : +- SubqueryBroadcast + : : +- BroadcastExchange + : : +- CometNativeColumnarToRow + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.date_dim + +- CometBroadcastExchange + +- CometFilter + +- CometNativeScan parquet spark_catalog.default.call_center -Comet accelerated 36 out of 97 eligible operators (37%). Final plan contains 21 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 75 out of 97 eligible operators (77%). Final plan contains 6 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q5a.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q5a.native_datafusion/extended.txt index ec2eef025a..9cc06fb361 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q5a.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q5a.native_datafusion/extended.txt @@ -1,322 +1,344 @@ CometNativeColumnarToRow +- CometTakeOrderedAndProject +- CometHashAggregate - +- CometColumnarExchange - +- HashAggregate - +- Union - :- HashAggregate - : +- CometNativeColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Union - : :- HashAggregate - : : +- CometNativeColumnarToRow - : : +- CometColumnarExchange - : : +- HashAggregate - : : +- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- Union - : : : : :- Project - : : : : : +- Filter - : : : : : +- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : +- Project - : : : : +- Filter - : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.store_returns [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- ReusedSubquery - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.store - : :- HashAggregate - : : +- CometNativeColumnarToRow - : : +- CometColumnarExchange - : : +- HashAggregate - : : +- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- Union - : : : : :- Project - : : : : : +- Filter - : : : : : +- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : +- ReusedSubquery - : : : : +- Project - : : : : +- Filter - : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.catalog_returns [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- ReusedSubquery - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.catalog_page - : +- HashAggregate - : +- CometNativeColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Union - : : : :- Project - : : : : +- Filter - : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- ReusedSubquery - : : : +- Project - : : : +- BroadcastHashJoin - : : : :- BroadcastExchange - : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.web_returns [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- ReusedSubquery - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- CometNativeColumnarToRow + +- CometExchange + +- CometHashAggregate + +- CometUnion + :- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometUnion + : :- CometHashAggregate + : : +- CometExchange + : : +- CometHashAggregate + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometUnion + : : : : :- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometNativeColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.store_returns + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometNativeColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.store + : :- CometHashAggregate + : : +- CometExchange + : : +- CometHashAggregate + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometUnion + : : : : :- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometNativeColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.catalog_returns + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometNativeColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.catalog_page + : +- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometUnion + : : : :- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.web_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometNativeColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometBroadcastExchange + : : : : +- CometNativeScan parquet spark_catalog.default.web_returns + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometNativeColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.web_sales + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.web_site + :- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometUnion + : :- CometHashAggregate + : : +- CometExchange + : : +- CometHashAggregate + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometUnion + : : : : :- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometNativeColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.store_returns + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometNativeColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.store + : :- CometHashAggregate + : : +- CometExchange + : : +- CometHashAggregate + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometUnion + : : : : :- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometNativeColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.catalog_returns + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometNativeColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.catalog_page + : +- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometUnion + : : : :- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.web_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometNativeColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometBroadcastExchange + : : : : +- CometNativeScan parquet spark_catalog.default.web_returns + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometNativeColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.web_sales + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange : +- CometProject : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.web_site - :- HashAggregate - : +- CometNativeColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- HashAggregate - : +- CometNativeColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Union - : :- HashAggregate - : : +- CometNativeColumnarToRow - : : +- CometColumnarExchange - : : +- HashAggregate - : : +- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- Union - : : : : :- Project - : : : : : +- Filter - : : : : : +- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : +- Project - : : : : +- Filter - : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.store_returns [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- ReusedSubquery - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.store - : :- HashAggregate - : : +- CometNativeColumnarToRow - : : +- CometColumnarExchange - : : +- HashAggregate - : : +- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- Union - : : : : :- Project - : : : : : +- Filter - : : : : : +- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : +- ReusedSubquery - : : : : +- Project - : : : : +- Filter - : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.catalog_returns [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- ReusedSubquery - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.catalog_page - : +- HashAggregate - : +- CometNativeColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Union - : : : :- Project - : : : : +- Filter - : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- ReusedSubquery - : : : +- Project - : : : +- BroadcastHashJoin - : : : :- BroadcastExchange - : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.web_returns [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- ReusedSubquery - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- CometNativeColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.web_site - +- HashAggregate - +- CometNativeColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- HashAggregate - +- CometNativeColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Union - :- HashAggregate - : +- CometNativeColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Union - : : : :- Project - : : : : +- Filter - : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- Project - : : : +- Filter - : : : +- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.store_returns [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : +- ReusedSubquery - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- CometNativeColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.store - :- HashAggregate - : +- CometNativeColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Union - : : : :- Project - : : : : +- Filter - : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- ReusedSubquery - : : : +- Project - : : : +- Filter - : : : +- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.catalog_returns [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : +- ReusedSubquery - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- CometNativeColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.catalog_page - +- HashAggregate - +- CometNativeColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- Union - : : :- Project - : : : +- Filter - : : : +- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : +- ReusedSubquery - : : +- Project - : : +- BroadcastHashJoin - : : :- BroadcastExchange - : : : +- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.web_returns [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : +- ReusedSubquery - : : +- CometNativeColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.web_sales - : +- BroadcastExchange - : +- CometNativeColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.date_dim - +- BroadcastExchange - +- CometNativeColumnarToRow - +- CometProject - +- CometFilter - +- CometNativeScan parquet spark_catalog.default.web_site + +- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometUnion + :- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometUnion + : : : :- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometNativeColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.store_returns + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometNativeColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.store + :- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometUnion + : : : :- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometNativeColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.catalog_returns + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometNativeColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.catalog_page + +- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometUnion + : : :- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.web_sales + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometNativeColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometBroadcastExchange + : : : +- CometNativeScan parquet spark_catalog.default.web_returns + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometNativeColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.web_sales + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.date_dim + +- CometBroadcastExchange + +- CometProject + +- CometFilter + +- CometNativeScan parquet spark_catalog.default.web_site -Comet accelerated 89 out of 263 eligible operators (33%). Final plan contains 57 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 287 out of 323 eligible operators (88%). Final plan contains 19 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q6.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q6.native_datafusion/extended.txt index f2cdb50e03..1cb21cb650 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q6.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q6.native_datafusion/extended.txt @@ -1,68 +1,57 @@ -TakeOrderedAndProject -+- Filter - +- HashAggregate - +- CometNativeColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.customer_address - : : : +- CometBroadcastExchange - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.customer - : : +- BroadcastExchange - : : +- Filter - : : +- ColumnarToRow - : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometProject - : : +- CometFilter - : : : +- Subquery - : : : +- CometNativeColumnarToRow - : : : +- CometHashAggregate - : : : +- CometExchange - : : : +- CometHashAggregate - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- CometNativeColumnarToRow - : +- CometProject - : +- CometFilter - : : +- Subquery - : : +- CometNativeColumnarToRow - : : +- CometHashAggregate - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- CometNativeScan parquet spark_catalog.default.date_dim - +- BroadcastExchange - +- CometNativeColumnarToRow - +- CometProject - +- CometBroadcastHashJoin - :- CometFilter - : +- CometNativeScan parquet spark_catalog.default.item - +- CometBroadcastExchange - +- CometFilter - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometFilter - +- CometNativeScan parquet spark_catalog.default.item +CometNativeColumnarToRow ++- CometTakeOrderedAndProject + +- CometFilter + +- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.customer_address + : : : +- CometBroadcastExchange + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.customer + : : +- CometBroadcastExchange + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : +- SubqueryBroadcast + : : +- BroadcastExchange + : : +- CometNativeColumnarToRow + : : +- CometProject + : : +- CometFilter + : : : +- Subquery + : : : +- CometNativeColumnarToRow + : : : +- CometHashAggregate + : : : +- CometExchange + : : : +- CometHashAggregate + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : : +- ReusedSubquery + : +- CometNativeScan parquet spark_catalog.default.date_dim + +- CometBroadcastExchange + +- CometProject + +- CometBroadcastHashJoin + :- CometFilter + : +- CometNativeScan parquet spark_catalog.default.item + +- CometBroadcastExchange + +- CometFilter + +- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometProject + +- CometFilter + +- CometNativeScan parquet spark_catalog.default.item -Comet accelerated 39 out of 58 eligible operators (67%). Final plan contains 8 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 48 out of 52 eligible operators (92%). Final plan contains 3 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q64.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q64.native_datafusion/extended.txt index 98db64e5b0..f1fe6fb3e3 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q64.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q64.native_datafusion/extended.txt @@ -40,22 +40,20 @@ CometNativeColumnarToRow : : : : : : : : : : : : : : : :- CometProject : : : : : : : : : : : : : : : : +- CometSortMergeJoin : : : : : : : : : : : : : : : : :- CometSort - : : : : : : : : : : : : : : : : : +- CometColumnarExchange - : : : : : : : : : : : : : : : : : +- Project - : : : : : : : : : : : : : : : : : +- BroadcastHashJoin - : : : : : : : : : : : : : : : : : :- BroadcastExchange - : : : : : : : : : : : : : : : : : : +- Filter - : : : : : : : : : : : : : : : : : : +- ColumnarToRow - : : : : : : : : : : : : : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : : : : : : : : : : : : : : +- SubqueryBroadcast - : : : : : : : : : : : : : : : : : : +- BroadcastExchange - : : : : : : : : : : : : : : : : : : +- CometNativeColumnarToRow - : : : : : : : : : : : : : : : : : : +- CometFilter - : : : : : : : : : : : : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : : : : : : : : : : : : : : +- CometNativeColumnarToRow - : : : : : : : : : : : : : : : : : +- CometProject - : : : : : : : : : : : : : : : : : +- CometFilter - : : : : : : : : : : : : : : : : : +- CometNativeScan parquet spark_catalog.default.store_returns + : : : : : : : : : : : : : : : : : +- CometExchange + : : : : : : : : : : : : : : : : : +- CometProject + : : : : : : : : : : : : : : : : : +- CometBroadcastHashJoin + : : : : : : : : : : : : : : : : : :- CometBroadcastExchange + : : : : : : : : : : : : : : : : : : +- CometFilter + : : : : : : : : : : : : : : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : : : : : : : : : : : : : : : +- SubqueryBroadcast + : : : : : : : : : : : : : : : : : : +- BroadcastExchange + : : : : : : : : : : : : : : : : : : +- CometNativeColumnarToRow + : : : : : : : : : : : : : : : : : : +- CometFilter + : : : : : : : : : : : : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : : : : : : : : : : : : +- CometProject + : : : : : : : : : : : : : : : : : +- CometFilter + : : : : : : : : : : : : : : : : : +- CometNativeScan parquet spark_catalog.default.store_returns : : : : : : : : : : : : : : : : +- CometSort : : : : : : : : : : : : : : : : +- CometProject : : : : : : : : : : : : : : : : +- CometFilter @@ -162,22 +160,20 @@ CometNativeColumnarToRow : : : : : : : : : : : : : : :- CometProject : : : : : : : : : : : : : : : +- CometSortMergeJoin : : : : : : : : : : : : : : : :- CometSort - : : : : : : : : : : : : : : : : +- CometColumnarExchange - : : : : : : : : : : : : : : : : +- Project - : : : : : : : : : : : : : : : : +- BroadcastHashJoin - : : : : : : : : : : : : : : : : :- BroadcastExchange - : : : : : : : : : : : : : : : : : +- Filter - : : : : : : : : : : : : : : : : : +- ColumnarToRow - : : : : : : : : : : : : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : : : : : : : : : : : : : +- SubqueryBroadcast - : : : : : : : : : : : : : : : : : +- BroadcastExchange - : : : : : : : : : : : : : : : : : +- CometNativeColumnarToRow - : : : : : : : : : : : : : : : : : +- CometFilter - : : : : : : : : : : : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : : : : : : : : : : : : : +- CometNativeColumnarToRow - : : : : : : : : : : : : : : : : +- CometProject - : : : : : : : : : : : : : : : : +- CometFilter - : : : : : : : : : : : : : : : : +- CometNativeScan parquet spark_catalog.default.store_returns + : : : : : : : : : : : : : : : : +- CometExchange + : : : : : : : : : : : : : : : : +- CometProject + : : : : : : : : : : : : : : : : +- CometBroadcastHashJoin + : : : : : : : : : : : : : : : : :- CometBroadcastExchange + : : : : : : : : : : : : : : : : : +- CometFilter + : : : : : : : : : : : : : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : : : : : : : : : : : : : : +- SubqueryBroadcast + : : : : : : : : : : : : : : : : : +- BroadcastExchange + : : : : : : : : : : : : : : : : : +- CometNativeColumnarToRow + : : : : : : : : : : : : : : : : : +- CometFilter + : : : : : : : : : : : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : : : : : : : : : : : +- CometProject + : : : : : : : : : : : : : : : : +- CometFilter + : : : : : : : : : : : : : : : : +- CometNativeScan parquet spark_catalog.default.store_returns : : : : : : : : : : : : : : : +- CometSort : : : : : : : : : : : : : : : +- CometProject : : : : : : : : : : : : : : : +- CometFilter @@ -248,4 +244,4 @@ CometNativeColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.item -Comet accelerated 228 out of 242 eligible operators (94%). Final plan contains 7 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 238 out of 242 eligible operators (98%). Final plan contains 3 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q67a.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q67a.native_datafusion/extended.txt index 84fdb89cb0..cdcf018198 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q67a.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q67a.native_datafusion/extended.txt @@ -3,345 +3,292 @@ TakeOrderedAndProject +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] +- CometNativeColumnarToRow +- CometSort - +- CometColumnarExchange - +- Union - :- HashAggregate - : +- CometNativeColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- Filter - : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.store - : +- BroadcastExchange - : +- CometNativeColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.item - :- HashAggregate - : +- CometNativeColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- HashAggregate - : +- CometNativeColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- Filter - : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.store - : +- BroadcastExchange - : +- CometNativeColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.item - :- HashAggregate - : +- CometNativeColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- HashAggregate - : +- CometNativeColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- Filter - : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.store - : +- BroadcastExchange - : +- CometNativeColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.item - :- HashAggregate - : +- CometNativeColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- HashAggregate - : +- CometNativeColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- Filter - : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.store - : +- BroadcastExchange - : +- CometNativeColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.item - :- HashAggregate - : +- CometNativeColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- HashAggregate - : +- CometNativeColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- Filter - : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.store - : +- BroadcastExchange - : +- CometNativeColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.item - :- HashAggregate - : +- CometNativeColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- HashAggregate - : +- CometNativeColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- Filter - : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.store - : +- BroadcastExchange - : +- CometNativeColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.item - :- HashAggregate - : +- CometNativeColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- HashAggregate - : +- CometNativeColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- Filter - : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.store - : +- BroadcastExchange - : +- CometNativeColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.item - :- HashAggregate - : +- CometNativeColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- HashAggregate - : +- CometNativeColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- Filter - : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.store - : +- BroadcastExchange - : +- CometNativeColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.item - +- HashAggregate - +- CometNativeColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- HashAggregate - +- CometNativeColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Filter - : : : +- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- CometNativeColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.store - +- BroadcastExchange - +- CometNativeColumnarToRow - +- CometProject - +- CometFilter - +- CometNativeScan parquet spark_catalog.default.item + +- CometExchange + +- CometUnion + :- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometNativeColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.store + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.item + :- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometNativeColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.store + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.item + :- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometNativeColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.store + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.item + :- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometNativeColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.store + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.item + :- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometNativeColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.store + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.item + :- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometNativeColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.store + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.item + :- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometNativeColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.store + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.item + :- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometNativeColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.store + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.item + +- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometNativeColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.store + +- CometBroadcastExchange + +- CometProject + +- CometFilter + +- CometNativeScan parquet spark_catalog.default.item -Comet accelerated 127 out of 282 eligible operators (45%). Final plan contains 63 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 261 out of 282 eligible operators (92%). Final plan contains 10 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q70a.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q70a.native_datafusion/extended.txt index ed925b771b..9f507b48bf 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q70a.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q70a.native_datafusion/extended.txt @@ -14,19 +14,18 @@ TakeOrderedAndProject : +- HashAggregate : +- Project : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Filter - : : : +- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow + : :- CometNativeColumnarToRow + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometNativeColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange : : +- CometProject : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -39,30 +38,32 @@ TakeOrderedAndProject : +- BroadcastExchange : +- Project : +- Filter - : +- Window - : +- Sort - : +- HashAggregate - : +- CometNativeColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Filter - : : : +- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : +- ReusedSubquery - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.store - : +- BroadcastExchange - : +- CometNativeColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] + : +- CometNativeColumnarToRow + : +- CometSort + : +- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometNativeColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.store + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.date_dim :- HashAggregate : +- CometNativeColumnarToRow : +- CometColumnarExchange @@ -73,19 +74,18 @@ TakeOrderedAndProject : +- HashAggregate : +- Project : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Filter - : : : +- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow + : :- CometNativeColumnarToRow + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometNativeColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange : : +- CometProject : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -98,30 +98,32 @@ TakeOrderedAndProject : +- BroadcastExchange : +- Project : +- Filter - : +- Window - : +- Sort - : +- HashAggregate - : +- CometNativeColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Filter - : : : +- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : +- ReusedSubquery - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.store - : +- BroadcastExchange - : +- CometNativeColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] + : +- CometNativeColumnarToRow + : +- CometSort + : +- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometNativeColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.store + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.date_dim +- HashAggregate +- CometNativeColumnarToRow +- CometColumnarExchange @@ -132,19 +134,18 @@ TakeOrderedAndProject +- HashAggregate +- Project +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- Filter - : : +- ColumnarToRow - : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- CometNativeColumnarToRow + :- CometNativeColumnarToRow + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : +- SubqueryBroadcast + : : +- BroadcastExchange + : : +- CometNativeColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange : +- CometProject : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -157,29 +158,31 @@ TakeOrderedAndProject +- BroadcastExchange +- Project +- Filter - +- Window - +- Sort - +- HashAggregate - +- CometNativeColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- Filter - : : +- ColumnarToRow - : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : +- ReusedSubquery - : +- BroadcastExchange - : +- CometNativeColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.store - +- BroadcastExchange - +- CometNativeColumnarToRow - +- CometProject - +- CometFilter - +- CometNativeScan parquet spark_catalog.default.date_dim + +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] + +- CometNativeColumnarToRow + +- CometSort + +- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : +- SubqueryBroadcast + : : +- BroadcastExchange + : : +- CometNativeColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.store + +- CometBroadcastExchange + +- CometProject + +- CometFilter + +- CometNativeScan parquet spark_catalog.default.date_dim -Comet accelerated 54 out of 153 eligible operators (35%). Final plan contains 30 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 111 out of 165 eligible operators (67%). Final plan contains 21 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q72.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q72.native_datafusion/extended.txt index 62c2b17b7e..623a68ddab 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q72.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q72.native_datafusion/extended.txt @@ -6,77 +6,67 @@ CometNativeColumnarToRow +- CometProject +- CometSortMergeJoin :- CometSort - : +- CometColumnarExchange - : +- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- Project - : : : : +- BroadcastHashJoin - : : : : :- Project - : : : : : +- BroadcastHashJoin - : : : : : :- Project - : : : : : : +- BroadcastHashJoin - : : : : : : :- Project - : : : : : : : +- BroadcastHashJoin - : : : : : : : :- Project - : : : : : : : : +- BroadcastHashJoin - : : : : : : : : :- Project - : : : : : : : : : +- BroadcastHashJoin - : : : : : : : : : :- Filter - : : : : : : : : : : +- ColumnarToRow - : : : : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : : : : : : +- SubqueryBroadcast - : : : : : : : : : : +- BroadcastExchange - : : : : : : : : : : +- CometNativeColumnarToRow - : : : : : : : : : : +- CometProject - : : : : : : : : : : +- CometFilter - : : : : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : : : : : : +- BroadcastExchange - : : : : : : : : : +- CometNativeColumnarToRow - : : : : : : : : : +- CometFilter - : : : : : : : : : +- CometNativeScan parquet spark_catalog.default.inventory - : : : : : : : : +- BroadcastExchange - : : : : : : : : +- CometNativeColumnarToRow - : : : : : : : : +- CometFilter - : : : : : : : : +- CometNativeScan parquet spark_catalog.default.warehouse - : : : : : : : +- BroadcastExchange - : : : : : : : +- CometNativeColumnarToRow - : : : : : : : +- CometFilter - : : : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : : : +- BroadcastExchange - : : : : : : +- CometNativeColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.customer_demographics - : : : : : +- BroadcastExchange - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.household_demographics - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- CometNativeColumnarToRow - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.promotion + : +- CometExchange + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometProject + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometProject + : : : : : : +- CometBroadcastHashJoin + : : : : : : :- CometProject + : : : : : : : +- CometBroadcastHashJoin + : : : : : : : :- CometProject + : : : : : : : : +- CometBroadcastHashJoin + : : : : : : : : :- CometProject + : : : : : : : : : +- CometBroadcastHashJoin + : : : : : : : : : :- CometFilter + : : : : : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : : : : : : : +- SubqueryBroadcast + : : : : : : : : : : +- BroadcastExchange + : : : : : : : : : : +- CometNativeColumnarToRow + : : : : : : : : : : +- CometProject + : : : : : : : : : : +- CometFilter + : : : : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : : : : +- CometBroadcastExchange + : : : : : : : : : +- CometFilter + : : : : : : : : : +- CometNativeScan parquet spark_catalog.default.inventory + : : : : : : : : +- CometBroadcastExchange + : : : : : : : : +- CometFilter + : : : : : : : : +- CometNativeScan parquet spark_catalog.default.warehouse + : : : : : : : +- CometBroadcastExchange + : : : : : : : +- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : : : : +- CometBroadcastExchange + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.customer_demographics + : : : : : +- CometBroadcastExchange + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.household_demographics + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.promotion +- CometSort +- CometExchange +- CometProject +- CometFilter +- CometNativeScan parquet spark_catalog.default.catalog_returns -Comet accelerated 37 out of 68 eligible operators (54%). Final plan contains 12 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 66 out of 68 eligible operators (97%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q74.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q74.native_datafusion/extended.txt index 5d218998be..51a3f5ca94 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q74.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q74.native_datafusion/extended.txt @@ -1,105 +1,98 @@ -TakeOrderedAndProject -+- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- BroadcastHashJoin - : : :- Filter - : : : +- HashAggregate - : : : +- CometNativeColumnarToRow - : : : +- CometColumnarExchange - : : : +- HashAggregate - : : : +- Project - : : : +- BroadcastHashJoin - : : : :- Project - : : : : +- BroadcastHashJoin - : : : : :- CometNativeColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.customer - : : : : +- BroadcastExchange - : : : : +- Filter - : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- HashAggregate - : : +- CometNativeColumnarToRow - : : +- CometColumnarExchange - : : +- HashAggregate - : : +- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.customer - : : : +- BroadcastExchange - : : : +- Filter - : : : +- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- Filter - : +- HashAggregate - : +- CometNativeColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.customer - : : +- BroadcastExchange - : : +- Filter - : : +- ColumnarToRow - : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : +- ReusedSubquery - : +- BroadcastExchange - : +- CometNativeColumnarToRow - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.date_dim - +- BroadcastExchange - +- HashAggregate - +- CometNativeColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- CometNativeColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.customer - : +- BroadcastExchange - : +- Filter - : +- ColumnarToRow - : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : +- ReusedSubquery - +- BroadcastExchange - +- CometNativeColumnarToRow - +- CometFilter - +- CometNativeScan parquet spark_catalog.default.date_dim +CometNativeColumnarToRow ++- CometTakeOrderedAndProject + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometHashAggregate + : : : +- CometExchange + : : : +- CometHashAggregate + : : : +- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.customer + : : : : +- CometBroadcastExchange + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometNativeColumnarToRow + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometHashAggregate + : : +- CometExchange + : : +- CometHashAggregate + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.customer + : : : +- CometBroadcastExchange + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometNativeColumnarToRow + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometFilter + : +- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.customer + : : +- CometBroadcastExchange + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.web_sales + : : +- SubqueryBroadcast + : : +- BroadcastExchange + : : +- CometNativeColumnarToRow + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.date_dim + +- CometBroadcastExchange + +- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.customer + : +- CometBroadcastExchange + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.web_sales + : +- SubqueryBroadcast + : +- BroadcastExchange + : +- CometNativeColumnarToRow + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.date_dim + +- CometBroadcastExchange + +- CometFilter + +- CometNativeScan parquet spark_catalog.default.date_dim -Comet accelerated 28 out of 85 eligible operators (32%). Final plan contains 18 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 83 out of 91 eligible operators (91%). Final plan contains 5 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q75.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q75.native_datafusion/extended.txt index 42106a0ffd..a943308f1e 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q75.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q75.native_datafusion/extended.txt @@ -15,28 +15,25 @@ CometNativeColumnarToRow : :- CometProject : : +- CometSortMergeJoin : : :- CometSort - : : : +- CometColumnarExchange - : : : +- Project - : : : +- BroadcastHashJoin - : : : :- Project - : : : : +- BroadcastHashJoin - : : : : :- Filter - : : : : : +- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometExchange + : : : +- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometNativeColumnarToRow + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : +- CometBroadcastExchange + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- CometSort : : +- CometExchange : : +- CometProject @@ -45,24 +42,25 @@ CometNativeColumnarToRow : :- CometProject : : +- CometSortMergeJoin : : :- CometSort - : : : +- CometColumnarExchange - : : : +- Project - : : : +- BroadcastHashJoin - : : : :- Project - : : : : +- BroadcastHashJoin - : : : : :- Filter - : : : : : +- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : +- ReusedSubquery - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometExchange + : : : +- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometNativeColumnarToRow + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.item + : : : +- CometBroadcastExchange + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- CometSort : : +- CometExchange : : +- CometProject @@ -71,24 +69,25 @@ CometNativeColumnarToRow : +- CometProject : +- CometSortMergeJoin : :- CometSort - : : +- CometColumnarExchange - : : +- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- Filter - : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- ReusedSubquery - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.item - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometExchange + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.web_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometNativeColumnarToRow + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.item + : : +- CometBroadcastExchange + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim : +- CometSort : +- CometExchange : +- CometProject @@ -107,28 +106,25 @@ CometNativeColumnarToRow :- CometProject : +- CometSortMergeJoin : :- CometSort - : : +- CometColumnarExchange - : : +- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- Filter - : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.item - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometExchange + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometNativeColumnarToRow + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.item + : : +- CometBroadcastExchange + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim : +- CometSort : +- CometExchange : +- CometProject @@ -137,24 +133,25 @@ CometNativeColumnarToRow :- CometProject : +- CometSortMergeJoin : :- CometSort - : : +- CometColumnarExchange - : : +- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- Filter - : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- ReusedSubquery - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.item - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometExchange + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometNativeColumnarToRow + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.item + : : +- CometBroadcastExchange + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim : +- CometSort : +- CometExchange : +- CometProject @@ -163,28 +160,29 @@ CometNativeColumnarToRow +- CometProject +- CometSortMergeJoin :- CometSort - : +- CometColumnarExchange - : +- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Filter - : : : +- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : +- ReusedSubquery - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.item - : +- BroadcastExchange - : +- CometNativeColumnarToRow - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometExchange + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.web_sales + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometNativeColumnarToRow + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.item + : +- CometBroadcastExchange + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.date_dim +- CometSort +- CometExchange +- CometProject +- CometFilter +- CometNativeScan parquet spark_catalog.default.web_returns -Comet accelerated 111 out of 167 eligible operators (66%). Final plan contains 21 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 167 out of 179 eligible operators (93%). Final plan contains 7 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q77a.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q77a.native_datafusion/extended.txt index 178b946ecb..24407fc0a2 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q77a.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q77a.native_datafusion/extended.txt @@ -9,132 +9,139 @@ CometNativeColumnarToRow : +- CometColumnarExchange : +- HashAggregate : +- Union + : :- CometNativeColumnarToRow + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometHashAggregate + : : : +- CometExchange + : : : +- CometHashAggregate + : : : +- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometNativeColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.store + : : +- CometBroadcastExchange + : : +- CometHashAggregate + : : +- CometExchange + : : +- CometHashAggregate + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.store_returns + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometNativeColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.store : :- Project - : : +- BroadcastHashJoin - : : :- HashAggregate - : : : +- CometNativeColumnarToRow - : : : +- CometColumnarExchange - : : : +- HashAggregate - : : : +- Project - : : : +- BroadcastHashJoin - : : : :- Project - : : : : +- BroadcastHashJoin - : : : : :- Filter - : : : : : +- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.store - : : +- BroadcastExchange - : : +- HashAggregate - : : +- CometNativeColumnarToRow - : : +- CometColumnarExchange - : : +- HashAggregate - : : +- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- Filter - : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.store_returns [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- ReusedSubquery - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.store - : :- Project - : : +- BroadcastNestedLoopJoin + : : +- BroadcastNestedLoopJoin [COMET: BroadcastNestedLoopJoin is not supported] : : :- BroadcastExchange - : : : +- HashAggregate - : : : +- CometNativeColumnarToRow - : : : +- CometColumnarExchange - : : : +- HashAggregate - : : : +- Project - : : : +- BroadcastHashJoin - : : : :- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- ReusedSubquery - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- HashAggregate - : : +- CometNativeColumnarToRow - : : +- CometColumnarExchange - : : +- HashAggregate - : : +- Project - : : +- BroadcastHashJoin - : : :- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.catalog_returns [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : +- ReusedSubquery - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- Project - : +- BroadcastHashJoin - : :- HashAggregate - : : +- CometNativeColumnarToRow - : : +- CometColumnarExchange - : : +- HashAggregate - : : +- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- Filter - : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- ReusedSubquery - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.web_page - : +- BroadcastExchange - : +- HashAggregate - : +- CometNativeColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Filter - : : : +- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.web_returns [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : +- ReusedSubquery - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- CometNativeColumnarToRow - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.web_page + : : : +- CometNativeColumnarToRow + : : : +- CometHashAggregate + : : : +- CometExchange + : : : +- CometHashAggregate + : : : +- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometNativeColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometNativeColumnarToRow + : : +- CometHashAggregate + : : +- CometExchange + : : +- CometHashAggregate + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometNativeScan parquet spark_catalog.default.catalog_returns + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometNativeColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometNativeColumnarToRow + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometHashAggregate + : : +- CometExchange + : : +- CometHashAggregate + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.web_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometNativeColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.web_page + : +- CometBroadcastExchange + : +- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.web_returns + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometNativeColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.web_page :- HashAggregate : +- CometNativeColumnarToRow : +- CometColumnarExchange @@ -144,132 +151,139 @@ CometNativeColumnarToRow : +- CometColumnarExchange : +- HashAggregate : +- Union + : :- CometNativeColumnarToRow + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometHashAggregate + : : : +- CometExchange + : : : +- CometHashAggregate + : : : +- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometNativeColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.store + : : +- CometBroadcastExchange + : : +- CometHashAggregate + : : +- CometExchange + : : +- CometHashAggregate + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.store_returns + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometNativeColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.store : :- Project - : : +- BroadcastHashJoin - : : :- HashAggregate - : : : +- CometNativeColumnarToRow - : : : +- CometColumnarExchange - : : : +- HashAggregate - : : : +- Project - : : : +- BroadcastHashJoin - : : : :- Project - : : : : +- BroadcastHashJoin - : : : : :- Filter - : : : : : +- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.store - : : +- BroadcastExchange - : : +- HashAggregate - : : +- CometNativeColumnarToRow - : : +- CometColumnarExchange - : : +- HashAggregate - : : +- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- Filter - : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.store_returns [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- ReusedSubquery - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.store - : :- Project - : : +- BroadcastNestedLoopJoin + : : +- BroadcastNestedLoopJoin [COMET: BroadcastNestedLoopJoin is not supported] : : :- BroadcastExchange - : : : +- HashAggregate - : : : +- CometNativeColumnarToRow - : : : +- CometColumnarExchange - : : : +- HashAggregate - : : : +- Project - : : : +- BroadcastHashJoin - : : : :- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- ReusedSubquery - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- HashAggregate - : : +- CometNativeColumnarToRow - : : +- CometColumnarExchange - : : +- HashAggregate - : : +- Project - : : +- BroadcastHashJoin - : : :- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.catalog_returns [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : +- ReusedSubquery - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- Project - : +- BroadcastHashJoin - : :- HashAggregate - : : +- CometNativeColumnarToRow - : : +- CometColumnarExchange - : : +- HashAggregate - : : +- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- Filter - : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- ReusedSubquery - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.web_page - : +- BroadcastExchange - : +- HashAggregate - : +- CometNativeColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Filter - : : : +- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.web_returns [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : +- ReusedSubquery - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- CometNativeColumnarToRow - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.web_page + : : : +- CometNativeColumnarToRow + : : : +- CometHashAggregate + : : : +- CometExchange + : : : +- CometHashAggregate + : : : +- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometNativeColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometNativeColumnarToRow + : : +- CometHashAggregate + : : +- CometExchange + : : +- CometHashAggregate + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometNativeScan parquet spark_catalog.default.catalog_returns + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometNativeColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometNativeColumnarToRow + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometHashAggregate + : : +- CometExchange + : : +- CometHashAggregate + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.web_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometNativeColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.web_page + : +- CometBroadcastExchange + : +- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.web_returns + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometNativeColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.web_page +- HashAggregate +- CometNativeColumnarToRow +- CometColumnarExchange @@ -279,131 +293,138 @@ CometNativeColumnarToRow +- CometColumnarExchange +- HashAggregate +- Union + :- CometNativeColumnarToRow + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometHashAggregate + : : +- CometExchange + : : +- CometHashAggregate + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometNativeColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.store + : +- CometBroadcastExchange + : +- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.store_returns + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometNativeColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.store :- Project - : +- BroadcastHashJoin - : :- HashAggregate - : : +- CometNativeColumnarToRow - : : +- CometColumnarExchange - : : +- HashAggregate - : : +- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- Filter - : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.store - : +- BroadcastExchange - : +- HashAggregate - : +- CometNativeColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Filter - : : : +- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.store_returns [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : +- ReusedSubquery - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- CometNativeColumnarToRow - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.store - :- Project - : +- BroadcastNestedLoopJoin + : +- BroadcastNestedLoopJoin [COMET: BroadcastNestedLoopJoin is not supported] : :- BroadcastExchange - : : +- HashAggregate - : : +- CometNativeColumnarToRow - : : +- CometColumnarExchange - : : +- HashAggregate - : : +- Project - : : +- BroadcastHashJoin - : : :- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : +- ReusedSubquery - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- HashAggregate - : +- CometNativeColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- ColumnarToRow - : : +- Scan parquet spark_catalog.default.catalog_returns [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : +- ReusedSubquery - : +- BroadcastExchange - : +- CometNativeColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.date_dim - +- Project - +- BroadcastHashJoin - :- HashAggregate - : +- CometNativeColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Filter - : : : +- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : +- ReusedSubquery - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- CometNativeColumnarToRow - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.web_page - +- BroadcastExchange - +- HashAggregate - +- CometNativeColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- Filter - : : +- ColumnarToRow - : : +- Scan parquet spark_catalog.default.web_returns [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : +- ReusedSubquery - : +- BroadcastExchange - : +- CometNativeColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.date_dim - +- BroadcastExchange - +- CometNativeColumnarToRow - +- CometFilter - +- CometNativeScan parquet spark_catalog.default.web_page + : : +- CometNativeColumnarToRow + : : +- CometHashAggregate + : : +- CometExchange + : : +- CometHashAggregate + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometNativeColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometNativeColumnarToRow + : +- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometNativeScan parquet spark_catalog.default.catalog_returns + : : +- SubqueryBroadcast + : : +- BroadcastExchange + : : +- CometNativeColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.date_dim + +- CometNativeColumnarToRow + +- CometProject + +- CometBroadcastHashJoin + :- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.web_sales + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometNativeColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.web_page + +- CometBroadcastExchange + +- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.web_returns + : : +- SubqueryBroadcast + : : +- BroadcastExchange + : : +- CometNativeColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.date_dim + +- CometBroadcastExchange + +- CometFilter + +- CometNativeScan parquet spark_catalog.default.web_page -Comet accelerated 113 out of 332 eligible operators (34%). Final plan contains 75 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 332 out of 392 eligible operators (84%). Final plan contains 36 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q78.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q78.native_datafusion/extended.txt index af4b7000b9..ebfdcc3c62 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q78.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q78.native_datafusion/extended.txt @@ -14,15 +14,14 @@ TakeOrderedAndProject : : : +- CometFilter : : : +- CometSortMergeJoin : : : :- CometSort - : : : : +- CometColumnarExchange - : : : : +- Filter - : : : : +- ColumnarToRow - : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometExchange + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometNativeColumnarToRow + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- CometSort : : : +- CometExchange : : : +- CometProject @@ -42,11 +41,14 @@ TakeOrderedAndProject : : +- CometFilter : : +- CometSortMergeJoin : : :- CometSort - : : : +- CometColumnarExchange - : : : +- Filter - : : : +- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : +- ReusedSubquery + : : : +- CometExchange + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.web_sales + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometNativeColumnarToRow + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- CometSort : : +- CometExchange : : +- CometProject @@ -66,11 +68,14 @@ TakeOrderedAndProject : +- CometFilter : +- CometSortMergeJoin : :- CometSort - : : +- CometColumnarExchange - : : +- Filter - : : +- ColumnarToRow - : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : +- ReusedSubquery + : : +- CometExchange + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : +- SubqueryBroadcast + : : +- BroadcastExchange + : : +- CometNativeColumnarToRow + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim : +- CometSort : +- CometExchange : +- CometProject @@ -80,4 +85,4 @@ TakeOrderedAndProject +- CometFilter +- CometNativeScan parquet spark_catalog.default.date_dim -Comet accelerated 64 out of 76 eligible operators (84%). Final plan contains 5 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 74 out of 82 eligible operators (90%). Final plan contains 4 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q80a.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q80a.native_datafusion/extended.txt index da58e863bd..ec566b42de 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q80a.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q80a.native_datafusion/extended.txt @@ -22,16 +22,15 @@ CometNativeColumnarToRow : : : : : :- CometProject : : : : : : +- CometSortMergeJoin : : : : : : :- CometSort - : : : : : : : +- CometColumnarExchange - : : : : : : : +- Filter - : : : : : : : +- ColumnarToRow - : : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : : : +- SubqueryBroadcast - : : : : : : : +- BroadcastExchange - : : : : : : : +- CometNativeColumnarToRow - : : : : : : : +- CometProject - : : : : : : : +- CometFilter - : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : : +- CometExchange + : : : : : : : +- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : : : : +- SubqueryBroadcast + : : : : : : : +- BroadcastExchange + : : : : : : : +- CometNativeColumnarToRow + : : : : : : : +- CometProject + : : : : : : : +- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : : : +- CometSort : : : : : : +- CometExchange : : : : : : +- CometProject @@ -67,11 +66,15 @@ CometNativeColumnarToRow : : : : : :- CometProject : : : : : : +- CometSortMergeJoin : : : : : : :- CometSort - : : : : : : : +- CometColumnarExchange - : : : : : : : +- Filter - : : : : : : : +- ColumnarToRow - : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : : : +- ReusedSubquery + : : : : : : : +- CometExchange + : : : : : : : +- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : : : : +- SubqueryBroadcast + : : : : : : : +- BroadcastExchange + : : : : : : : +- CometNativeColumnarToRow + : : : : : : : +- CometProject + : : : : : : : +- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : : : +- CometSort : : : : : : +- CometExchange : : : : : : +- CometProject @@ -107,11 +110,15 @@ CometNativeColumnarToRow : : : : :- CometProject : : : : : +- CometSortMergeJoin : : : : : :- CometSort - : : : : : : +- CometColumnarExchange - : : : : : : +- Filter - : : : : : : +- ColumnarToRow - : : : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : : +- ReusedSubquery + : : : : : : +- CometExchange + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.web_sales + : : : : : : +- SubqueryBroadcast + : : : : : : +- BroadcastExchange + : : : : : : +- CometNativeColumnarToRow + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : : +- CometSort : : : : : +- CometExchange : : : : : +- CometProject @@ -154,16 +161,15 @@ CometNativeColumnarToRow : : : : : :- CometProject : : : : : : +- CometSortMergeJoin : : : : : : :- CometSort - : : : : : : : +- CometColumnarExchange - : : : : : : : +- Filter - : : : : : : : +- ColumnarToRow - : : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : : : +- SubqueryBroadcast - : : : : : : : +- BroadcastExchange - : : : : : : : +- CometNativeColumnarToRow - : : : : : : : +- CometProject - : : : : : : : +- CometFilter - : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : : +- CometExchange + : : : : : : : +- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : : : : +- SubqueryBroadcast + : : : : : : : +- BroadcastExchange + : : : : : : : +- CometNativeColumnarToRow + : : : : : : : +- CometProject + : : : : : : : +- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : : : +- CometSort : : : : : : +- CometExchange : : : : : : +- CometProject @@ -199,11 +205,15 @@ CometNativeColumnarToRow : : : : : :- CometProject : : : : : : +- CometSortMergeJoin : : : : : : :- CometSort - : : : : : : : +- CometColumnarExchange - : : : : : : : +- Filter - : : : : : : : +- ColumnarToRow - : : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : : : +- ReusedSubquery + : : : : : : : +- CometExchange + : : : : : : : +- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : : : : +- SubqueryBroadcast + : : : : : : : +- BroadcastExchange + : : : : : : : +- CometNativeColumnarToRow + : : : : : : : +- CometProject + : : : : : : : +- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : : : +- CometSort : : : : : : +- CometExchange : : : : : : +- CometProject @@ -239,11 +249,15 @@ CometNativeColumnarToRow : : : : :- CometProject : : : : : +- CometSortMergeJoin : : : : : :- CometSort - : : : : : : +- CometColumnarExchange - : : : : : : +- Filter - : : : : : : +- ColumnarToRow - : : : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : : +- ReusedSubquery + : : : : : : +- CometExchange + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.web_sales + : : : : : : +- SubqueryBroadcast + : : : : : : +- BroadcastExchange + : : : : : : +- CometNativeColumnarToRow + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : : +- CometSort : : : : : +- CometExchange : : : : : +- CometProject @@ -286,16 +300,15 @@ CometNativeColumnarToRow : : : : :- CometProject : : : : : +- CometSortMergeJoin : : : : : :- CometSort - : : : : : : +- CometColumnarExchange - : : : : : : +- Filter - : : : : : : +- ColumnarToRow - : : : : : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometNativeColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : +- CometExchange + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : : : +- SubqueryBroadcast + : : : : : : +- BroadcastExchange + : : : : : : +- CometNativeColumnarToRow + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : : +- CometSort : : : : : +- CometExchange : : : : : +- CometProject @@ -331,11 +344,15 @@ CometNativeColumnarToRow : : : : :- CometProject : : : : : +- CometSortMergeJoin : : : : : :- CometSort - : : : : : : +- CometColumnarExchange - : : : : : : +- Filter - : : : : : : +- ColumnarToRow - : : : : : : +- Scan parquet spark_catalog.default.catalog_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : : +- ReusedSubquery + : : : : : : +- CometExchange + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : : : +- SubqueryBroadcast + : : : : : : +- BroadcastExchange + : : : : : : +- CometNativeColumnarToRow + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : : +- CometSort : : : : : +- CometExchange : : : : : +- CometProject @@ -371,11 +388,15 @@ CometNativeColumnarToRow : : : :- CometProject : : : : +- CometSortMergeJoin : : : : :- CometSort - : : : : : +- CometColumnarExchange - : : : : : +- Filter - : : : : : +- ColumnarToRow - : : : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : : : +- ReusedSubquery + : : : : : +- CometExchange + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.web_sales + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometNativeColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : +- CometSort : : : : +- CometExchange : : : : +- CometProject @@ -398,4 +419,4 @@ CometNativeColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.promotion -Comet accelerated 356 out of 386 eligible operators (92%). Final plan contains 13 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 392 out of 410 eligible operators (95%). Final plan contains 10 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q86a.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q86a.native_datafusion/extended.txt index d56e25c3af..c1df720935 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q86a.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q86a.native_datafusion/extended.txt @@ -5,97 +5,83 @@ TakeOrderedAndProject +- CometSort +- CometExchange +- CometHashAggregate - +- CometColumnarExchange - +- HashAggregate - +- Union - :- HashAggregate - : +- CometNativeColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Filter - : : : +- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- CometNativeColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.item - :- HashAggregate - : +- CometNativeColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- HashAggregate - : +- CometNativeColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Filter - : : : +- ColumnarToRow - : : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- CometNativeColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.item - +- HashAggregate - +- CometNativeColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- HashAggregate - +- CometNativeColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- Filter - : : +- ColumnarToRow - : : +- Scan parquet spark_catalog.default.web_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- CometNativeColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.date_dim - +- BroadcastExchange - +- CometNativeColumnarToRow - +- CometProject - +- CometFilter - +- CometNativeScan parquet spark_catalog.default.item + +- CometExchange + +- CometHashAggregate + +- CometUnion + :- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.web_sales + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometNativeColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.item + :- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.web_sales + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometNativeColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.item + +- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.web_sales + : : +- SubqueryBroadcast + : : +- BroadcastExchange + : : +- CometNativeColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.date_dim + +- CometBroadcastExchange + +- CometProject + +- CometFilter + +- CometNativeScan parquet spark_catalog.default.item -Comet accelerated 36 out of 81 eligible operators (44%). Final plan contains 18 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 72 out of 81 eligible operators (88%). Final plan contains 4 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q98.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q98.native_datafusion/extended.txt index b8d60879d2..6c2b043008 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q98.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q98.native_datafusion/extended.txt @@ -5,33 +5,29 @@ CometNativeColumnarToRow +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] +- CometNativeColumnarToRow +- CometSort - +- CometColumnarExchange - +- HashAggregate - +- CometNativeColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- Filter - : : +- ColumnarToRow - : : +- Scan parquet spark_catalog.default.store_sales [COMET: Native DataFusion scan does not support subqueries/dynamic pruning] - : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- CometNativeColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.item - +- BroadcastExchange - +- CometNativeColumnarToRow - +- CometProject - +- CometFilter - +- CometNativeScan parquet spark_catalog.default.date_dim + +- CometExchange + +- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : +- SubqueryBroadcast + : : +- BroadcastExchange + : : +- CometNativeColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.item + +- CometBroadcastExchange + +- CometProject + +- CometFilter + +- CometNativeScan parquet spark_catalog.default.date_dim -Comet accelerated 14 out of 28 eligible operators (50%). Final plan contains 7 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 24 out of 28 eligible operators (85%). Final plan contains 3 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/scala/org/apache/spark/sql/comet/CometPlanStabilitySuite.scala b/spark/src/test/scala/org/apache/spark/sql/comet/CometPlanStabilitySuite.scala index 69e52bd31a..9b4ad4d358 100644 --- a/spark/src/test/scala/org/apache/spark/sql/comet/CometPlanStabilitySuite.scala +++ b/spark/src/test/scala/org/apache/spark/sql/comet/CometPlanStabilitySuite.scala @@ -218,7 +218,9 @@ trait CometPlanStabilitySuite extends DisableAdaptiveExecutionSuite with TPCDSBa CometConf.COMET_ENABLED.key -> "true", CometConf.COMET_NATIVE_SCAN_ENABLED.key -> "true", CometConf.COMET_EXEC_ENABLED.key -> "true", - CometConf.COMET_DPP_FALLBACK_ENABLED.key -> "false", + // Non-AQE DPP is natively supported. AQE is disabled in this suite + // (DisableAdaptiveExecutionSuite), so all DPP is non-AQE and works natively. + CometConf.COMET_DPP_FALLBACK_ENABLED.key -> "true", CometConf.COMET_EXEC_SHUFFLE_ENABLED.key -> "true", CometConf.COMET_EXEC_SORT_MERGE_JOIN_WITH_JOIN_FILTER_ENABLED.key -> "true", // as well as for v1.4/q9, v1.4/q44, v2.7.0/q6, v2.7.0/q64 From 17fa823827ce0aa09221136cf145cc49e0a91118 Mon Sep 17 00:00:00 2001 From: Matt Butrovich Date: Mon, 20 Apr 2026 21:35:01 -0400 Subject: [PATCH 05/21] Remove DPP config. --- .../src/main/scala/org/apache/comet/CometConf.scala | 8 -------- .../org/apache/comet/rules/CometScanRule.scala | 3 +-- .../shuffle/CometShuffleExchangeExec.scala | 9 +++++---- .../org/apache/comet/exec/CometExecSuite.scala | 13 ++++--------- .../sql/comet/CometDppFallbackRepro3949Suite.scala | 5 +---- .../spark/sql/comet/CometPlanStabilitySuite.scala | 3 --- .../comet/CometShuffleFallbackStickinessSuite.scala | 9 +++------ 7 files changed, 14 insertions(+), 36 deletions(-) diff --git a/common/src/main/scala/org/apache/comet/CometConf.scala b/common/src/main/scala/org/apache/comet/CometConf.scala index 2395e63081..8b1dd9b044 100644 --- a/common/src/main/scala/org/apache/comet/CometConf.scala +++ b/common/src/main/scala/org/apache/comet/CometConf.scala @@ -554,14 +554,6 @@ object CometConf extends ShimCometConf { .doubleConf .createWithDefault(1.0) - val COMET_DPP_FALLBACK_ENABLED: ConfigEntry[Boolean] = - conf("spark.comet.dppFallback.enabled") - .category(CATEGORY_EXEC) - .doc("Whether to fall back to Spark for queries that use AQE Dynamic Partition Pruning " + - "(SubqueryAdaptiveBroadcastExec). Non-AQE DPP is always supported natively.") - .booleanConf - .createWithDefault(true) - val COMET_DEBUG_ENABLED: ConfigEntry[Boolean] = conf("spark.comet.debug.enabled") .category(CATEGORY_EXEC) diff --git a/spark/src/main/scala/org/apache/comet/rules/CometScanRule.scala b/spark/src/main/scala/org/apache/comet/rules/CometScanRule.scala index 6ae453fa68..befff09bc0 100644 --- a/spark/src/main/scala/org/apache/comet/rules/CometScanRule.scala +++ b/spark/src/main/scala/org/apache/comet/rules/CometScanRule.scala @@ -139,8 +139,7 @@ case class CometScanRule(session: SparkSession) private def transformV1Scan(plan: SparkPlan, scanExec: FileSourceScanExec): SparkPlan = { - if (COMET_DPP_FALLBACK_ENABLED.get() && - scanExec.partitionFilters.exists(isAqeDynamicPruningFilter)) { + if (scanExec.partitionFilters.exists(isAqeDynamicPruningFilter)) { return withInfo(scanExec, "AQE Dynamic Partition Pruning is not supported") } diff --git a/spark/src/main/scala/org/apache/spark/sql/comet/execution/shuffle/CometShuffleExchangeExec.scala b/spark/src/main/scala/org/apache/spark/sql/comet/execution/shuffle/CometShuffleExchangeExec.scala index bd6922b9e8..9d12846642 100644 --- a/spark/src/main/scala/org/apache/spark/sql/comet/execution/shuffle/CometShuffleExchangeExec.scala +++ b/spark/src/main/scala/org/apache/spark/sql/comet/execution/shuffle/CometShuffleExchangeExec.scala @@ -275,10 +275,11 @@ object CometShuffleExchangeExec case None => } - // DPP fallback is a combined-path decision: a Comet shuffle wrapped around a stage that - // still contains a DPP scan produces inefficient row<->columnar transitions. Disqualifies - // both paths. - if (CometConf.COMET_DPP_FALLBACK_ENABLED.get() && stageContainsDPPScan(s)) { + // A Comet shuffle wrapped around a stage that still contains a Spark FileSourceScanExec + // with DPP produces inefficient row<->columnar transitions. This only happens when the + // scan fell back (e.g., AQE DPP not supported). If the scan converted to + // CometNativeScanExec, stageContainsDPPScan won't match (it checks FileSourceScanExec). + if (stageContainsDPPScan(s)) { withInfos(s, Set("Stage contains a scan with Dynamic Partition Pruning")) return None } diff --git a/spark/src/test/scala/org/apache/comet/exec/CometExecSuite.scala b/spark/src/test/scala/org/apache/comet/exec/CometExecSuite.scala index 868f8b5244..b3ca0c52ff 100644 --- a/spark/src/test/scala/org/apache/comet/exec/CometExecSuite.scala +++ b/spark/src/test/scala/org/apache/comet/exec/CometExecSuite.scala @@ -128,9 +128,7 @@ class CometExecSuite extends CometTestBase { // note that this test does not trigger DPP with v2 data source Seq("parquet").foreach { v1List => - withSQLConf( - SQLConf.USE_V1_SOURCE_LIST.key -> v1List, - CometConf.COMET_DPP_FALLBACK_ENABLED.key -> "true") { + withSQLConf(SQLConf.USE_V1_SOURCE_LIST.key -> v1List) { spark.read.parquet(factPath).createOrReplaceTempView("dpp_fact") spark.read.parquet(dimPath).createOrReplaceTempView("dpp_dim") val df = @@ -164,8 +162,7 @@ class CometExecSuite extends CometTestBase { Seq("parquet").foreach { v1List => withSQLConf( SQLConf.USE_V1_SOURCE_LIST.key -> v1List, - SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> "-1", - CometConf.COMET_DPP_FALLBACK_ENABLED.key -> "true") { + SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> "-1") { spark.read.parquet(factPath).createOrReplaceTempView("dpp_fact2") spark.read.parquet(dimPath).createOrReplaceTempView("dpp_dim2") val df = @@ -202,8 +199,7 @@ class CometExecSuite extends CometTestBase { // with SubqueryBroadcastExec, not SubqueryAdaptiveBroadcastExec withSQLConf( SQLConf.USE_V1_SOURCE_LIST.key -> "parquet", - SQLConf.ADAPTIVE_EXECUTION_ENABLED.key -> "false", - CometConf.COMET_DPP_FALLBACK_ENABLED.key -> "true") { + SQLConf.ADAPTIVE_EXECUTION_ENABLED.key -> "false") { spark.read.parquet(factPath).createOrReplaceTempView("dpp_fact_bhj") spark.read.parquet(dimPath).createOrReplaceTempView("dpp_dim_bhj") val df = spark.sql( @@ -249,8 +245,7 @@ class CometExecSuite extends CometTestBase { withSQLConf( SQLConf.USE_V1_SOURCE_LIST.key -> "parquet", SQLConf.ADAPTIVE_EXECUTION_ENABLED.key -> "false", - SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> "-1", - CometConf.COMET_DPP_FALLBACK_ENABLED.key -> "true") { + SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> "-1") { spark.read.parquet(factPath).createOrReplaceTempView("dpp_fact_smj") spark.read.parquet(dimPath).createOrReplaceTempView("dpp_dim_smj") val df = spark.sql( diff --git a/spark/src/test/scala/org/apache/spark/sql/comet/CometDppFallbackRepro3949Suite.scala b/spark/src/test/scala/org/apache/spark/sql/comet/CometDppFallbackRepro3949Suite.scala index 955d900888..1e0dae391d 100644 --- a/spark/src/test/scala/org/apache/spark/sql/comet/CometDppFallbackRepro3949Suite.scala +++ b/spark/src/test/scala/org/apache/spark/sql/comet/CometDppFallbackRepro3949Suite.scala @@ -109,7 +109,6 @@ class CometDppFallbackRepro3949Suite extends CometTestBase { withTempDir { dir => buildDppTables(dir, "mech") withSQLConf( - CometConf.COMET_DPP_FALLBACK_ENABLED.key -> "true", SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> "-1", SQLConf.PREFER_SORTMERGEJOIN.key -> "true", SQLConf.ADAPTIVE_EXECUTION_ENABLED.key -> "true", @@ -336,9 +335,7 @@ class CometDppFallbackRepro3949Suite extends CometTestBase { val suspicious = mutable.Buffer.empty[(String, Int, String)] for ((variantName, variantConf) <- variants; (q, idx) <- queries.zipWithIndex) { - val conf = variantConf ++ Map( - CometConf.COMET_DPP_FALLBACK_ENABLED.key -> "true", - SQLConf.USE_V1_SOURCE_LIST.key -> "parquet") + val conf = variantConf ++ Map(SQLConf.USE_V1_SOURCE_LIST.key -> "parquet") try { withSQLConf(conf.toSeq: _*) { val df = spark.sql(q) diff --git a/spark/src/test/scala/org/apache/spark/sql/comet/CometPlanStabilitySuite.scala b/spark/src/test/scala/org/apache/spark/sql/comet/CometPlanStabilitySuite.scala index 9b4ad4d358..fe4ce4cde9 100644 --- a/spark/src/test/scala/org/apache/spark/sql/comet/CometPlanStabilitySuite.scala +++ b/spark/src/test/scala/org/apache/spark/sql/comet/CometPlanStabilitySuite.scala @@ -218,9 +218,6 @@ trait CometPlanStabilitySuite extends DisableAdaptiveExecutionSuite with TPCDSBa CometConf.COMET_ENABLED.key -> "true", CometConf.COMET_NATIVE_SCAN_ENABLED.key -> "true", CometConf.COMET_EXEC_ENABLED.key -> "true", - // Non-AQE DPP is natively supported. AQE is disabled in this suite - // (DisableAdaptiveExecutionSuite), so all DPP is non-AQE and works natively. - CometConf.COMET_DPP_FALLBACK_ENABLED.key -> "true", CometConf.COMET_EXEC_SHUFFLE_ENABLED.key -> "true", CometConf.COMET_EXEC_SORT_MERGE_JOIN_WITH_JOIN_FILTER_ENABLED.key -> "true", // as well as for v1.4/q9, v1.4/q44, v2.7.0/q6, v2.7.0/q64 diff --git a/spark/src/test/scala/org/apache/spark/sql/comet/CometShuffleFallbackStickinessSuite.scala b/spark/src/test/scala/org/apache/spark/sql/comet/CometShuffleFallbackStickinessSuite.scala index 0374c57068..4302ab1391 100644 --- a/spark/src/test/scala/org/apache/spark/sql/comet/CometShuffleFallbackStickinessSuite.scala +++ b/spark/src/test/scala/org/apache/spark/sql/comet/CometShuffleFallbackStickinessSuite.scala @@ -52,11 +52,9 @@ class CometShuffleFallbackStickinessSuite extends CometTestBase { val shuffle = ShuffleExchangeExec(SinglePartition, SyntheticLeaf(Nil)) withInfo(shuffle, "pretend prior pass decided Spark fallback") - withSQLConf(CometConf.COMET_DPP_FALLBACK_ENABLED.key -> "true") { - assert( - CometShuffleExchangeExec.shuffleSupported(shuffle).isEmpty, - "marked shuffle must preserve its prior-pass fallback decision") - } + assert( + CometShuffleExchangeExec.shuffleSupported(shuffle).isEmpty, + "marked shuffle must preserve its prior-pass fallback decision") } test( @@ -85,7 +83,6 @@ class CometShuffleFallbackStickinessSuite extends CometTestBase { spark.read.parquet(dimPath).createOrReplaceTempView("t_sticky_dim") withSQLConf( - CometConf.COMET_DPP_FALLBACK_ENABLED.key -> "true", SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> "-1", SQLConf.PREFER_SORTMERGEJOIN.key -> "true", SQLConf.ADAPTIVE_EXECUTION_ENABLED.key -> "true", From 71327c1a7ea44f3d11a62a2b6242363b7978dea6 Mon Sep 17 00:00:00 2001 From: Matt Butrovich Date: Mon, 20 Apr 2026 22:36:14 -0400 Subject: [PATCH 06/21] Update Spark diffs. --- dev/diffs/3.4.3.diff | 28 ++++++++++++-------- dev/diffs/3.5.8.diff | 61 ++++++++++++++++++++++++++++---------------- dev/diffs/4.0.1.diff | 30 +++++++++++++--------- 3 files changed, 74 insertions(+), 45 deletions(-) diff --git a/dev/diffs/3.4.3.diff b/dev/diffs/3.4.3.diff index 4312bb36cc..3d542c4a4c 100644 --- a/dev/diffs/3.4.3.diff +++ b/dev/diffs/3.4.3.diff @@ -417,28 +417,31 @@ index daef11ae4d6..9f3cc9181f2 100644 assert(exchanges.size == 2) } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DynamicPartitionPruningSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DynamicPartitionPruningSuite.scala -index f33432ddb6f..7d758d2481f 100644 +index f33432ddb6f..8ec7285ed84 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/DynamicPartitionPruningSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/DynamicPartitionPruningSuite.scala @@ -22,6 +22,7 @@ import org.scalatest.GivenWhenThen import org.apache.spark.sql.catalyst.expressions.{DynamicPruningExpression, Expression} import org.apache.spark.sql.catalyst.expressions.CodegenObjectFactoryMode._ import org.apache.spark.sql.catalyst.plans.ExistenceJoin -+import org.apache.spark.sql.comet.CometScanExec ++import org.apache.spark.sql.comet.{CometNativeScanExec, CometScanExec} import org.apache.spark.sql.connector.catalog.{InMemoryTableCatalog, InMemoryTableWithV2FilterCatalog} import org.apache.spark.sql.execution._ import org.apache.spark.sql.execution.adaptive._ -@@ -262,6 +263,9 @@ abstract class DynamicPartitionPruningSuiteBase +@@ -262,6 +263,12 @@ abstract class DynamicPartitionPruningSuiteBase case s: BatchScanExec => s.runtimeFilters.collect { case d: DynamicPruningExpression => d.child } + case s: CometScanExec => s.partitionFilters.collect { + case d: DynamicPruningExpression => d.child ++ } ++ case s: CometNativeScanExec => s.partitionFilters.collect { ++ case d: DynamicPruningExpression => d.child + } case _ => Nil } } -@@ -1027,7 +1032,8 @@ abstract class DynamicPartitionPruningSuiteBase +@@ -1027,7 +1034,8 @@ abstract class DynamicPartitionPruningSuiteBase } } @@ -448,7 +451,7 @@ index f33432ddb6f..7d758d2481f 100644 withSQLConf(SQLConf.DYNAMIC_PARTITION_PRUNING_REUSE_BROADCAST_ONLY.key -> "false", SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> "-1") { withTable("large", "dimTwo", "dimThree") { -@@ -1215,7 +1221,8 @@ abstract class DynamicPartitionPruningSuiteBase +@@ -1215,7 +1223,8 @@ abstract class DynamicPartitionPruningSuiteBase } test("SPARK-32509: Unused Dynamic Pruning filter shouldn't affect " + @@ -458,7 +461,7 @@ index f33432ddb6f..7d758d2481f 100644 withSQLConf(SQLConf.DYNAMIC_PARTITION_PRUNING_REUSE_BROADCAST_ONLY.key -> "true") { withSQLConf(SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> "-1") { val df = sql( -@@ -1423,7 +1430,8 @@ abstract class DynamicPartitionPruningSuiteBase +@@ -1423,7 +1432,8 @@ abstract class DynamicPartitionPruningSuiteBase } } @@ -468,7 +471,7 @@ index f33432ddb6f..7d758d2481f 100644 withSQLConf(SQLConf.DYNAMIC_PARTITION_PRUNING_REUSE_BROADCAST_ONLY.key -> "true") { val df = sql( """ WITH v as ( -@@ -1698,7 +1706,8 @@ abstract class DynamicPartitionPruningV1Suite extends DynamicPartitionPruningDat +@@ -1698,7 +1708,8 @@ abstract class DynamicPartitionPruningV1Suite extends DynamicPartitionPruningDat * Check the static scan metrics with and without DPP */ test("static scan metrics", @@ -478,7 +481,7 @@ index f33432ddb6f..7d758d2481f 100644 withSQLConf(SQLConf.DYNAMIC_PARTITION_PRUNING_ENABLED.key -> "true", SQLConf.DYNAMIC_PARTITION_PRUNING_REUSE_BROADCAST_ONLY.key -> "false", SQLConf.EXCHANGE_REUSE_ENABLED.key -> "false") { -@@ -1729,6 +1738,8 @@ abstract class DynamicPartitionPruningV1Suite extends DynamicPartitionPruningDat +@@ -1729,6 +1740,8 @@ abstract class DynamicPartitionPruningV1Suite extends DynamicPartitionPruningDat case s: BatchScanExec => // we use f1 col for v2 tables due to schema pruning s.output.exists(_.exists(_.argString(maxFields = 100).contains("f1"))) @@ -933,7 +936,7 @@ index b5b34922694..a72403780c4 100644 protected val baseResourcePath = { // use the same way as `SQLQueryTestSuite` to get the resource path diff --git a/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala -index 525d97e4998..843f0472c23 100644 +index 525d97e4998..8a3e7457618 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala @@ -1508,7 +1508,8 @@ class SQLQuerySuite extends QueryTest with SharedSparkSession with AdaptiveSpark @@ -3056,7 +3059,7 @@ index 1510e8957f9..7618419d8ff 100644 implicit val formats = new DefaultFormats { override def dateFormatter = new SimpleDateFormat("yyyy-MM-dd'T'HH:mm:ss") diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/DynamicPartitionPruningHiveScanSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/DynamicPartitionPruningHiveScanSuite.scala -index 52abd248f3a..7a199931a08 100644 +index 52abd248f3a..b4e096cae24 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/DynamicPartitionPruningHiveScanSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/DynamicPartitionPruningHiveScanSuite.scala @@ -19,6 +19,7 @@ package org.apache.spark.sql.hive @@ -3067,12 +3070,15 @@ index 52abd248f3a..7a199931a08 100644 import org.apache.spark.sql.execution._ import org.apache.spark.sql.execution.adaptive.{DisableAdaptiveExecutionSuite, EnableAdaptiveExecutionSuite} import org.apache.spark.sql.hive.execution.HiveTableScanExec -@@ -35,6 +36,9 @@ abstract class DynamicPartitionPruningHiveScanSuiteBase +@@ -35,6 +36,12 @@ abstract class DynamicPartitionPruningHiveScanSuiteBase case s: FileSourceScanExec => s.partitionFilters.collect { case d: DynamicPruningExpression => d.child } + case s: CometScanExec => s.partitionFilters.collect { + case d: DynamicPruningExpression => d.child ++ } ++ case s: CometNativeScanExec => s.partitionFilters.collect { ++ case d: DynamicPruningExpression => d.child + } case h: HiveTableScanExec => h.partitionPruningPred.collect { case d: DynamicPruningExpression => d.child diff --git a/dev/diffs/3.5.8.diff b/dev/diffs/3.5.8.diff index e349a94f2d..6462f4f489 100644 --- a/dev/diffs/3.5.8.diff +++ b/dev/diffs/3.5.8.diff @@ -398,28 +398,31 @@ index c4fb4fa943c..a04b23870a8 100644 assert(exchanges.size == 2) } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DynamicPartitionPruningSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DynamicPartitionPruningSuite.scala -index f33432ddb6f..7122af0d414 100644 +index f33432ddb6f..8ec7285ed84 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/DynamicPartitionPruningSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/DynamicPartitionPruningSuite.scala @@ -22,6 +22,7 @@ import org.scalatest.GivenWhenThen import org.apache.spark.sql.catalyst.expressions.{DynamicPruningExpression, Expression} import org.apache.spark.sql.catalyst.expressions.CodegenObjectFactoryMode._ import org.apache.spark.sql.catalyst.plans.ExistenceJoin -+import org.apache.spark.sql.comet.CometScanExec ++import org.apache.spark.sql.comet.{CometNativeScanExec, CometScanExec} import org.apache.spark.sql.connector.catalog.{InMemoryTableCatalog, InMemoryTableWithV2FilterCatalog} import org.apache.spark.sql.execution._ import org.apache.spark.sql.execution.adaptive._ -@@ -262,6 +263,9 @@ abstract class DynamicPartitionPruningSuiteBase +@@ -262,6 +263,12 @@ abstract class DynamicPartitionPruningSuiteBase case s: BatchScanExec => s.runtimeFilters.collect { case d: DynamicPruningExpression => d.child } + case s: CometScanExec => s.partitionFilters.collect { + case d: DynamicPruningExpression => d.child ++ } ++ case s: CometNativeScanExec => s.partitionFilters.collect { ++ case d: DynamicPruningExpression => d.child + } case _ => Nil } } -@@ -1027,7 +1031,8 @@ abstract class DynamicPartitionPruningSuiteBase +@@ -1027,7 +1034,8 @@ abstract class DynamicPartitionPruningSuiteBase } } @@ -429,7 +432,7 @@ index f33432ddb6f..7122af0d414 100644 withSQLConf(SQLConf.DYNAMIC_PARTITION_PRUNING_REUSE_BROADCAST_ONLY.key -> "false", SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> "-1") { withTable("large", "dimTwo", "dimThree") { -@@ -1215,7 +1220,8 @@ abstract class DynamicPartitionPruningSuiteBase +@@ -1215,7 +1223,8 @@ abstract class DynamicPartitionPruningSuiteBase } test("SPARK-32509: Unused Dynamic Pruning filter shouldn't affect " + @@ -439,7 +442,7 @@ index f33432ddb6f..7122af0d414 100644 withSQLConf(SQLConf.DYNAMIC_PARTITION_PRUNING_REUSE_BROADCAST_ONLY.key -> "true") { withSQLConf(SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> "-1") { val df = sql( -@@ -1423,7 +1429,8 @@ abstract class DynamicPartitionPruningSuiteBase +@@ -1423,7 +1432,8 @@ abstract class DynamicPartitionPruningSuiteBase } } @@ -449,7 +452,7 @@ index f33432ddb6f..7122af0d414 100644 withSQLConf(SQLConf.DYNAMIC_PARTITION_PRUNING_REUSE_BROADCAST_ONLY.key -> "true") { val df = sql( """ WITH v as ( -@@ -1698,7 +1705,8 @@ abstract class DynamicPartitionPruningV1Suite extends DynamicPartitionPruningDat +@@ -1698,7 +1708,8 @@ abstract class DynamicPartitionPruningV1Suite extends DynamicPartitionPruningDat * Check the static scan metrics with and without DPP */ test("static scan metrics", @@ -459,7 +462,7 @@ index f33432ddb6f..7122af0d414 100644 withSQLConf(SQLConf.DYNAMIC_PARTITION_PRUNING_ENABLED.key -> "true", SQLConf.DYNAMIC_PARTITION_PRUNING_REUSE_BROADCAST_ONLY.key -> "false", SQLConf.EXCHANGE_REUSE_ENABLED.key -> "false") { -@@ -1729,6 +1737,8 @@ abstract class DynamicPartitionPruningV1Suite extends DynamicPartitionPruningDat +@@ -1729,6 +1740,8 @@ abstract class DynamicPartitionPruningV1Suite extends DynamicPartitionPruningDat case s: BatchScanExec => // we use f1 col for v2 tables due to schema pruning s.output.exists(_.exists(_.argString(maxFields = 100).contains("f1"))) @@ -928,7 +931,7 @@ index c26757c9cff..d55775f09d7 100644 protected val baseResourcePath = { // use the same way as `SQLQueryTestSuite` to get the resource path diff --git a/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala -index 3cf2bfd17ab..49728c35c42 100644 +index 3cf2bfd17ab..b1c1e41e6a9 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala @@ -1521,7 +1521,8 @@ class SQLQuerySuite extends QueryTest with SharedSparkSession with AdaptiveSpark @@ -1245,16 +1248,17 @@ index de24b8c82b0..1f835481290 100644 setupTestData() diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/RemoveRedundantProjectsSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/RemoveRedundantProjectsSuite.scala -index 9e9d717db3b..73de2b84938 100644 +index 9e9d717db3b..cdd1042a880 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/RemoveRedundantProjectsSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/RemoveRedundantProjectsSuite.scala @@ -17,7 +17,10 @@ package org.apache.spark.sql.execution +-import org.apache.spark.sql.{DataFrame, QueryTest, Row} +import org.apache.comet.CometConf + - import org.apache.spark.sql.{DataFrame, QueryTest, Row} ++import org.apache.spark.sql.{DataFrame, IgnoreCometNativeDataFusion, QueryTest, Row} +import org.apache.spark.sql.comet.CometProjectExec import org.apache.spark.sql.connector.SimpleWritableDataSource import org.apache.spark.sql.execution.adaptive.{AdaptiveSparkPlanHelper, DisableAdaptiveExecutionSuite, EnableAdaptiveExecutionSuite} @@ -1271,7 +1275,17 @@ index 9e9d717db3b..73de2b84938 100644 assert(actual == expected) } } -@@ -134,12 +140,26 @@ abstract class RemoveRedundantProjectsSuiteBase +@@ -112,7 +118,8 @@ abstract class RemoveRedundantProjectsSuiteBase + assertProjectExec(query, 1, 3) + } + +- test("join with ordering requirement") { ++ test("join with ordering requirement", ++ IgnoreCometNativeDataFusion("https://github.com/apache/datafusion-comet/issues/4014")) { + val query = "select * from (select key, a, c, b from testView) as t1 join " + + "(select key, a, b, c from testView) as t2 on t1.key = t2.key where t2.a > 50" + assertProjectExec(query, 2, 2) +@@ -134,12 +141,26 @@ abstract class RemoveRedundantProjectsSuiteBase val df = data.selectExpr("a", "b", "key", "explode(array(key, a, b)) as d").filter("d > 0") df.collect() val plan = df.queryExecution.executedPlan @@ -1300,7 +1314,7 @@ index 9e9d717db3b..73de2b84938 100644 case g @ GenerateExec(_, requiredChildOutput, _, _, child) => g.copy(requiredChildOutput = requiredChildOutput.reverse, child = ProjectExec(requiredChildOutput.reverse, child)) -@@ -151,6 +171,7 @@ abstract class RemoveRedundantProjectsSuiteBase +@@ -151,6 +172,7 @@ abstract class RemoveRedundantProjectsSuiteBase // The manually added ProjectExec node shouldn't be removed. assert(collectWithSubqueries(newExecutedPlan) { case p: ProjectExec => p @@ -1420,7 +1434,7 @@ index 5a413c77754..207b66e1d7b 100644 import testImplicits._ diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/adaptive/AdaptiveQueryExecSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/adaptive/AdaptiveQueryExecSuite.scala -index 2f8e401e743..a4f94417dcc 100644 +index 2f8e401e743..dbcf3171946 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/adaptive/AdaptiveQueryExecSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/adaptive/AdaptiveQueryExecSuite.scala @@ -27,9 +27,11 @@ import org.scalatest.time.SpanSugar._ @@ -1782,7 +1796,7 @@ index 2f8e401e743..a4f94417dcc 100644 CostEvaluator.instantiate( classOf[SimpleShuffleSortCostEvaluator].getCanonicalName, spark.sparkContext.getConf) intercept[IllegalArgumentException] { -@@ -2452,6 +2493,7 @@ class AdaptiveQueryExecSuite +@@ -2452,6 +2492,7 @@ class AdaptiveQueryExecSuite val (_, adaptive) = runAdaptiveAndVerifyResult(query) assert(adaptive.collect { case sort: SortExec => sort @@ -1790,7 +1804,7 @@ index 2f8e401e743..a4f94417dcc 100644 }.size == 1) val read = collect(adaptive) { case read: AQEShuffleReadExec => read -@@ -2469,7 +2511,8 @@ class AdaptiveQueryExecSuite +@@ -2469,7 +2510,8 @@ class AdaptiveQueryExecSuite } } @@ -1800,7 +1814,7 @@ index 2f8e401e743..a4f94417dcc 100644 withTempView("v") { withSQLConf( SQLConf.ADAPTIVE_OPTIMIZE_SKEWS_IN_REBALANCE_PARTITIONS_ENABLED.key -> "true", -@@ -2581,7 +2624,7 @@ class AdaptiveQueryExecSuite +@@ -2581,7 +2623,7 @@ class AdaptiveQueryExecSuite runAdaptiveAndVerifyResult("SELECT key1 FROM skewData1 JOIN skewData2 ON key1 = key2 " + "JOIN skewData3 ON value2 = value3") val shuffles1 = collect(adaptive1) { @@ -1809,7 +1823,7 @@ index 2f8e401e743..a4f94417dcc 100644 } assert(shuffles1.size == 4) val smj1 = findTopLevelSortMergeJoin(adaptive1) -@@ -2592,7 +2635,7 @@ class AdaptiveQueryExecSuite +@@ -2592,7 +2634,7 @@ class AdaptiveQueryExecSuite runAdaptiveAndVerifyResult("SELECT key1 FROM skewData1 JOIN skewData2 ON key1 = key2 " + "JOIN skewData3 ON value1 = value3") val shuffles2 = collect(adaptive2) { @@ -1818,7 +1832,7 @@ index 2f8e401e743..a4f94417dcc 100644 } assert(shuffles2.size == 4) val smj2 = findTopLevelSortMergeJoin(adaptive2) -@@ -2850,6 +2893,7 @@ class AdaptiveQueryExecSuite +@@ -2850,6 +2892,7 @@ class AdaptiveQueryExecSuite }.size == (if (firstAccess) 1 else 0)) assert(collect(initialExecutedPlan) { case s: SortExec => s @@ -1826,7 +1840,7 @@ index 2f8e401e743..a4f94417dcc 100644 }.size == (if (firstAccess) 2 else 0)) assert(collect(initialExecutedPlan) { case i: InMemoryTableScanLike => i -@@ -2980,7 +3024,9 @@ class AdaptiveQueryExecSuite +@@ -2980,7 +3023,9 @@ class AdaptiveQueryExecSuite val plan = df.queryExecution.executedPlan.asInstanceOf[AdaptiveSparkPlanExec] assert(plan.inputPlan.isInstanceOf[TakeOrderedAndProjectExec]) @@ -2973,7 +2987,7 @@ index c63c748953f..7edca9c93a6 100644 implicit val formats = new DefaultFormats { override def dateFormatter = new SimpleDateFormat("yyyy-MM-dd'T'HH:mm:ss") diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/DynamicPartitionPruningHiveScanSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/DynamicPartitionPruningHiveScanSuite.scala -index 52abd248f3a..7a199931a08 100644 +index 52abd248f3a..b4e096cae24 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/DynamicPartitionPruningHiveScanSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/DynamicPartitionPruningHiveScanSuite.scala @@ -19,6 +19,7 @@ package org.apache.spark.sql.hive @@ -2984,12 +2998,15 @@ index 52abd248f3a..7a199931a08 100644 import org.apache.spark.sql.execution._ import org.apache.spark.sql.execution.adaptive.{DisableAdaptiveExecutionSuite, EnableAdaptiveExecutionSuite} import org.apache.spark.sql.hive.execution.HiveTableScanExec -@@ -35,6 +36,9 @@ abstract class DynamicPartitionPruningHiveScanSuiteBase +@@ -35,6 +36,12 @@ abstract class DynamicPartitionPruningHiveScanSuiteBase case s: FileSourceScanExec => s.partitionFilters.collect { case d: DynamicPruningExpression => d.child } + case s: CometScanExec => s.partitionFilters.collect { + case d: DynamicPruningExpression => d.child ++ } ++ case s: CometNativeScanExec => s.partitionFilters.collect { ++ case d: DynamicPruningExpression => d.child + } case h: HiveTableScanExec => h.partitionPruningPred.collect { case d: DynamicPruningExpression => d.child diff --git a/dev/diffs/4.0.1.diff b/dev/diffs/4.0.1.diff index bd42aaa3e0..e88165da6d 100644 --- a/dev/diffs/4.0.1.diff +++ b/dev/diffs/4.0.1.diff @@ -589,28 +589,31 @@ index 81713c777bc..b5f92ed9742 100644 assert(exchanges.size == 2) } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DynamicPartitionPruningSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DynamicPartitionPruningSuite.scala -index 2c24cc7d570..dae4419bd1c 100644 +index 2c24cc7d570..697ea7f8a60 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/DynamicPartitionPruningSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/DynamicPartitionPruningSuite.scala @@ -22,6 +22,7 @@ import org.scalatest.GivenWhenThen import org.apache.spark.sql.catalyst.expressions.{DynamicPruningExpression, Expression} import org.apache.spark.sql.catalyst.expressions.CodegenObjectFactoryMode._ import org.apache.spark.sql.catalyst.plans.ExistenceJoin -+import org.apache.spark.sql.comet.CometScanExec ++import org.apache.spark.sql.comet.{CometNativeScanExec, CometScanExec} import org.apache.spark.sql.connector.catalog.{InMemoryTableCatalog, InMemoryTableWithV2FilterCatalog} import org.apache.spark.sql.execution._ import org.apache.spark.sql.execution.adaptive._ -@@ -262,6 +263,9 @@ abstract class DynamicPartitionPruningSuiteBase +@@ -262,6 +263,12 @@ abstract class DynamicPartitionPruningSuiteBase case s: BatchScanExec => s.runtimeFilters.collect { case d: DynamicPruningExpression => d.child } + case s: CometScanExec => s.partitionFilters.collect { + case d: DynamicPruningExpression => d.child ++ } ++ case s: CometNativeScanExec => s.partitionFilters.collect { ++ case d: DynamicPruningExpression => d.child + } case _ => Nil } } -@@ -1027,7 +1031,8 @@ abstract class DynamicPartitionPruningSuiteBase +@@ -1027,7 +1034,8 @@ abstract class DynamicPartitionPruningSuiteBase } } @@ -620,7 +623,7 @@ index 2c24cc7d570..dae4419bd1c 100644 withSQLConf(SQLConf.DYNAMIC_PARTITION_PRUNING_REUSE_BROADCAST_ONLY.key -> "false", SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> "-1") { withTable("large", "dimTwo", "dimThree") { -@@ -1151,7 +1156,8 @@ abstract class DynamicPartitionPruningSuiteBase +@@ -1151,7 +1159,8 @@ abstract class DynamicPartitionPruningSuiteBase } } @@ -630,7 +633,7 @@ index 2c24cc7d570..dae4419bd1c 100644 withSQLConf(SQLConf.DYNAMIC_PARTITION_PRUNING_REUSE_BROADCAST_ONLY.key -> "true", SQLConf.ADAPTIVE_OPTIMIZER_EXCLUDED_RULES.key -> AQEPropagateEmptyRelation.ruleName, SQLConf.ANSI_ENABLED.key -> "false" // ANSI mode doesn't support "String + String" -@@ -1215,7 +1221,8 @@ abstract class DynamicPartitionPruningSuiteBase +@@ -1215,7 +1224,8 @@ abstract class DynamicPartitionPruningSuiteBase } test("SPARK-32509: Unused Dynamic Pruning filter shouldn't affect " + @@ -640,7 +643,7 @@ index 2c24cc7d570..dae4419bd1c 100644 withSQLConf(SQLConf.DYNAMIC_PARTITION_PRUNING_REUSE_BROADCAST_ONLY.key -> "true") { withSQLConf(SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> "-1") { val df = sql( -@@ -1330,6 +1337,7 @@ abstract class DynamicPartitionPruningSuiteBase +@@ -1330,6 +1340,7 @@ abstract class DynamicPartitionPruningSuiteBase } test("Subquery reuse across the whole plan", @@ -648,7 +651,7 @@ index 2c24cc7d570..dae4419bd1c 100644 DisableAdaptiveExecution("DPP in AQE must reuse broadcast")) { withSQLConf(SQLConf.DYNAMIC_PARTITION_PRUNING_ENABLED.key -> "true", SQLConf.DYNAMIC_PARTITION_PRUNING_REUSE_BROADCAST_ONLY.key -> "false", -@@ -1424,7 +1432,8 @@ abstract class DynamicPartitionPruningSuiteBase +@@ -1424,7 +1435,8 @@ abstract class DynamicPartitionPruningSuiteBase } } @@ -658,7 +661,7 @@ index 2c24cc7d570..dae4419bd1c 100644 withSQLConf(SQLConf.DYNAMIC_PARTITION_PRUNING_REUSE_BROADCAST_ONLY.key -> "true") { val df = sql( """ WITH v as ( -@@ -1699,7 +1708,8 @@ abstract class DynamicPartitionPruningV1Suite extends DynamicPartitionPruningDat +@@ -1699,7 +1711,8 @@ abstract class DynamicPartitionPruningV1Suite extends DynamicPartitionPruningDat * Check the static scan metrics with and without DPP */ test("static scan metrics", @@ -668,7 +671,7 @@ index 2c24cc7d570..dae4419bd1c 100644 withSQLConf(SQLConf.DYNAMIC_PARTITION_PRUNING_ENABLED.key -> "true", SQLConf.DYNAMIC_PARTITION_PRUNING_REUSE_BROADCAST_ONLY.key -> "false", SQLConf.EXCHANGE_REUSE_ENABLED.key -> "false") { -@@ -1730,6 +1740,8 @@ abstract class DynamicPartitionPruningV1Suite extends DynamicPartitionPruningDat +@@ -1730,6 +1743,8 @@ abstract class DynamicPartitionPruningV1Suite extends DynamicPartitionPruningDat case s: BatchScanExec => // we use f1 col for v2 tables due to schema pruning s.output.exists(_.exists(_.argString(maxFields = 100).contains("f1"))) @@ -3864,7 +3867,7 @@ index 982d57fb287..6017f36c440 100644 implicit val formats: DefaultFormats = new DefaultFormats { override def dateFormatter = new SimpleDateFormat("yyyy-MM-dd'T'HH:mm:ss") diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/DynamicPartitionPruningHiveScanSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/DynamicPartitionPruningHiveScanSuite.scala -index 52abd248f3a..7a199931a08 100644 +index 52abd248f3a..b4e096cae24 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/DynamicPartitionPruningHiveScanSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/DynamicPartitionPruningHiveScanSuite.scala @@ -19,6 +19,7 @@ package org.apache.spark.sql.hive @@ -3875,12 +3878,15 @@ index 52abd248f3a..7a199931a08 100644 import org.apache.spark.sql.execution._ import org.apache.spark.sql.execution.adaptive.{DisableAdaptiveExecutionSuite, EnableAdaptiveExecutionSuite} import org.apache.spark.sql.hive.execution.HiveTableScanExec -@@ -35,6 +36,9 @@ abstract class DynamicPartitionPruningHiveScanSuiteBase +@@ -35,6 +36,12 @@ abstract class DynamicPartitionPruningHiveScanSuiteBase case s: FileSourceScanExec => s.partitionFilters.collect { case d: DynamicPruningExpression => d.child } + case s: CometScanExec => s.partitionFilters.collect { + case d: DynamicPruningExpression => d.child ++ } ++ case s: CometNativeScanExec => s.partitionFilters.collect { ++ case d: DynamicPruningExpression => d.child + } case h: HiveTableScanExec => h.partitionPruningPred.collect { case d: DynamicPruningExpression => d.child From 03530e9c7031b1a6a05eb0d90dd8d2d4287d6cbb Mon Sep 17 00:00:00 2001 From: Matt Butrovich Date: Mon, 20 Apr 2026 22:50:10 -0400 Subject: [PATCH 07/21] Update docs. --- docs/source/contributor-guide/parquet_scans.md | 2 +- docs/source/contributor-guide/roadmap.md | 10 +++++----- 2 files changed, 6 insertions(+), 6 deletions(-) diff --git a/docs/source/contributor-guide/parquet_scans.md b/docs/source/contributor-guide/parquet_scans.md index 134d4c7119..cb94c28514 100644 --- a/docs/source/contributor-guide/parquet_scans.md +++ b/docs/source/contributor-guide/parquet_scans.md @@ -46,7 +46,7 @@ The following features are not supported by either scan implementation, and Come - Spark's Datasource V2 API. When `spark.sql.sources.useV1SourceList` does not include `parquet`, Spark uses the V2 API for Parquet scans. The DataFusion-based implementations only support the V1 API. - Spark metadata columns (e.g., `_metadata.file_path`) -- No support for Dynamic Partition Pruning (DPP) +- No support for AQE Dynamic Partition Pruning (DPP). Non-AQE DPP is supported. The following shared limitation may produce incorrect results without falling back to Spark: diff --git a/docs/source/contributor-guide/roadmap.md b/docs/source/contributor-guide/roadmap.md index c685e0fb80..3f9fc7eebc 100644 --- a/docs/source/contributor-guide/roadmap.md +++ b/docs/source/contributor-guide/roadmap.md @@ -42,14 +42,14 @@ more Spark SQL tests and fully implementing ANSI support ([#313]) for all suppor ### Dynamic Partition Pruning -Iceberg table scans support Dynamic Partition Pruning (DPP) filters generated by Spark's `PlanDynamicPruningFilters` -optimizer rule ([#3349]). However, we still need to bring this functionality to our Parquet reader. Furthermore, -Spark's `PlanAdaptiveDynamicPruningFilters` optimizer rule runs after Comet's rules, so DPP with Adaptive Query -Execution requires a redesign of Comet's plan translation. We are focused on implementing DPP to keep Comet competitive -with benchmarks that benefit from this feature like TPC-DS. This effort can be tracked at [#3510]. +Both Iceberg table scans and Parquet V1 native scans (`CometNativeScanExec`) support non-AQE Dynamic Partition Pruning +(DPP) filters generated by Spark's `PlanDynamicPruningFilters` optimizer rule ([#3349], [#3511]). However, Spark's +`PlanAdaptiveDynamicPruningFilters` optimizer rule runs after Comet's rules, so DPP with Adaptive Query Execution +requires a redesign of Comet's plan translation. This effort can be tracked at [#3510]. [#3349]: https://github.com/apache/datafusion-comet/pull/3349 [#3510]: https://github.com/apache/datafusion-comet/issues/3510 +[#3511]: https://github.com/apache/datafusion-comet/pull/3511 ## Ongoing Improvements From 6bcc2c31d6cd4c2ac462017baeb66e4b3c59e941 Mon Sep 17 00:00:00 2001 From: Matt Butrovich Date: Tue, 21 Apr 2026 09:12:33 -0400 Subject: [PATCH 08/21] Passes tests with reuse. --- .../apache/comet/rules/CometExecRule.scala | 99 ++++++++++++++++++- .../spark/sql/comet/CometNativeScanExec.scala | 41 +++++++- .../comet/shims/ShimSubqueryBroadcast.scala | 7 +- .../comet/shims/ShimSubqueryBroadcast.scala | 7 +- .../comet/shims/ShimSubqueryBroadcast.scala | 7 +- .../apache/comet/exec/CometExecSuite.scala | 75 +++++++++++++- 6 files changed, 229 insertions(+), 7 deletions(-) diff --git a/spark/src/main/scala/org/apache/comet/rules/CometExecRule.scala b/spark/src/main/scala/org/apache/comet/rules/CometExecRule.scala index c0c5c602cc..8d87c9765b 100644 --- a/spark/src/main/scala/org/apache/comet/rules/CometExecRule.scala +++ b/spark/src/main/scala/org/apache/comet/rules/CometExecRule.scala @@ -53,6 +53,7 @@ import org.apache.comet.CometSparkSessionExtensions._ import org.apache.comet.rules.CometExecRule.allExecs import org.apache.comet.serde._ import org.apache.comet.serde.operator._ +import org.apache.comet.shims.ShimSubqueryBroadcast object CometExecRule { @@ -93,7 +94,9 @@ object CometExecRule { /** * Spark physical optimizer rule for replacing Spark operators with Comet operators. */ -case class CometExecRule(session: SparkSession) extends Rule[SparkPlan] { +case class CometExecRule(session: SparkSession) + extends Rule[SparkPlan] + with ShimSubqueryBroadcast { private lazy val showTransformations = CometConf.COMET_EXPLAIN_TRANSFORMATIONS.get() @@ -298,8 +301,100 @@ case class CometExecRule(session: SparkSession) extends Rule[SparkPlan] { } } + // scalastyle:off println plan.transformUp { case op => - convertNode(op) + val hasSubqueryExpr = op.expressions.exists(_.exists { + case _: InSubqueryExec => true + case _ => false + }) + if (hasSubqueryExpr) { + println(s"[RULE-DEBUG] convertNode on ${op.getClass.getSimpleName} " + + s"which HAS InSubqueryExec expressions") + op.expressions.foreach { expr => + expr.foreach { + case sub: InSubqueryExec => + println(s"[RULE-DEBUG] InSubqueryExec.plan: ${sub.plan.getClass.getSimpleName}") + sub.plan match { + case sb: SubqueryBroadcastExec => + println(s"[RULE-DEBUG] SubqueryBroadcast.child: " + + s"${sb.child.getClass.getSimpleName}") + sb.child match { + case b: BroadcastExchangeExec => + println(s"[RULE-DEBUG] BroadcastExchange.child: " + + s"${b.child.getClass.getSimpleName}") + println(s"[RULE-DEBUG] BroadcastExchange.child is CometNative? " + + s"${b.child.isInstanceOf[CometNativeExec]}") + println(s"[RULE-DEBUG] BroadcastExchange.children all CometNative? " + + s"${b.children.forall(_.isInstanceOf[CometNativeExec])}") + case other => + println(s"[RULE-DEBUG] SubqueryBroadcast.child is: " + + s"${other.getClass.getSimpleName}") + } + case other => + println(s"[RULE-DEBUG] sub.plan is: ${other.getClass.getSimpleName}") + } + case _ => + } + } + } + val converted = convertNode(op) + // Replace SubqueryBroadcastExec with CometSubqueryBroadcastExec in DPP expressions + // when the broadcast child has a Comet plan underneath. This enables exchange reuse + // between the DPP subquery and the join's CometBroadcastExchangeExec because both + // will have the same CometBroadcastExchangeExec type and canonical form. + convertSubqueryBroadcasts(converted) + } + // scalastyle:on println + } + + /** + * Replace SubqueryBroadcastExec with CometSubqueryBroadcastExec in a node's expressions. + * + * When CometExecRule converts BroadcastExchangeExec to CometBroadcastExchangeExec on the + * join side, the DPP subquery still references the original BroadcastExchangeExec. + * ReuseExchangeAndSubquery (which runs after Comet rules) can't match them because they + * have different types. By replacing SubqueryBroadcastExec with CometSubqueryBroadcastExec + * (which wraps a CometBroadcastExchangeExec), both sides have the same exchange type and + * reuse works. + * + * The BroadcastExchangeExec in the subquery has a CometNativeColumnarToRowExec child + * (inserted by ApplyColumnarRulesAndInsertTransitions because BroadcastExchangeExec expects + * row input). We strip this transition and create CometBroadcastExchangeExec with the + * underlying Comet plan directly. + */ + private def convertSubqueryBroadcasts(plan: SparkPlan): SparkPlan = { + plan.transformExpressionsUp { + case inSub: InSubqueryExec => + inSub.plan match { + case sub: SubqueryBroadcastExec => + sub.child match { + case b: BroadcastExchangeExec => + // The BroadcastExchangeExec child is CometNativeColumnarToRowExec wrapping + // a Comet plan. Strip the row transition to get the columnar Comet plan. + val cometChild = b.child match { + case c2r: CometNativeColumnarToRowExec => c2r.child + case other => other + } + if (cometChild.isInstanceOf[CometNativeExec]) { + // scalastyle:off println + println(s"[RULE-DEBUG] Converting SubqueryBroadcastExec to " + + s"CometSubqueryBroadcastExec, cometChild=${cometChild.getClass.getSimpleName}") + // scalastyle:on println + val cometBroadcast = CometBroadcastExchangeExec( + b, b.output, b.mode, cometChild) + val cometSub = CometSubqueryBroadcastExec( + sub.name, + getSubqueryBroadcastExecIndices(sub), + sub.buildKeys, + cometBroadcast) + inSub.withNewPlan(cometSub) + } else { + inSub + } + case _ => inSub + } + case _ => inSub + } } } diff --git a/spark/src/main/scala/org/apache/spark/sql/comet/CometNativeScanExec.scala b/spark/src/main/scala/org/apache/spark/sql/comet/CometNativeScanExec.scala index 71b018c728..05219512d2 100644 --- a/spark/src/main/scala/org/apache/spark/sql/comet/CometNativeScanExec.scala +++ b/spark/src/main/scala/org/apache/spark/sql/comet/CometNativeScanExec.scala @@ -176,15 +176,54 @@ case class CometNativeScanExec( * partition's files (lazily, as tasks are scheduled). */ @transient private lazy val serializedPartitionData: (Array[Byte], Array[Array[Byte]]) = { + // scalastyle:off println + println(s"[DPP-DEBUG] serializedPartitionData: checking partitionFilters") + partitionFilters.foreach { + case DynamicPruningExpression(e: InSubqueryExec) => + println(s"[DPP-DEBUG] InSubqueryExec plan=${e.plan.getClass.getSimpleName} " + + s"values empty=${e.values().isEmpty}") + case other => + println(s"[DPP-DEBUG] filter: ${other.getClass.getSimpleName}") + } + // scalastyle:on println // Ensure DPP subqueries are resolved before accessing file partitions. // serializedPartitionData can be triggered from findAllPlanData (via commonData) on a // different execution path than the standard prepare() -> executeSubqueries() flow // (e.g., from a BroadcastExchangeExec thread). We must resolve DPP here explicitly. partitionFilters.foreach { case DynamicPruningExpression(e: InSubqueryExec) if e.values().isEmpty => - e.updateResult() + // scalastyle:off println + println(s"[DPP-DEBUG] calling updateResult on InSubqueryExec " + + s"plan=${e.plan.getClass.getSimpleName} id=${System.identityHashCode(e)}") + // scalastyle:on println + try { + e.updateResult() + // scalastyle:off println + println(s"[DPP-DEBUG] updateResult succeeded, values empty=${e.values().isEmpty}") + // scalastyle:on println + } catch { + // scalastyle:off println + case ex: Exception => + println(s"[DPP-DEBUG] updateResult FAILED: ${ex.getMessage}") + throw ex + // scalastyle:on println + } case _ => } + // Also resolve DPP in CometScanExec's partitionFilters, which may reference + // a different InSubqueryExec instance (with the original SubqueryBroadcastExec). + // CometScanExec.dynamicallySelectedPartitions evaluates these filters. + if (scan != null) { + scan.partitionFilters.foreach { + case DynamicPruningExpression(e: InSubqueryExec) if e.values().isEmpty => + // scalastyle:off println + println(s"[DPP-DEBUG] also resolving scan's InSubqueryExec " + + s"plan=${e.plan.getClass.getSimpleName} id=${System.identityHashCode(e)}") + // scalastyle:on println + e.updateResult() + case _ => + } + } // Extract common data from nativeOp val commonBytes = nativeOp.getNativeScan.getCommon.toByteArray diff --git a/spark/src/main/spark-3.4/org/apache/comet/shims/ShimSubqueryBroadcast.scala b/spark/src/main/spark-3.4/org/apache/comet/shims/ShimSubqueryBroadcast.scala index 1ff0935041..292ed2cb18 100644 --- a/spark/src/main/spark-3.4/org/apache/comet/shims/ShimSubqueryBroadcast.scala +++ b/spark/src/main/spark-3.4/org/apache/comet/shims/ShimSubqueryBroadcast.scala @@ -19,7 +19,7 @@ package org.apache.comet.shims -import org.apache.spark.sql.execution.SubqueryAdaptiveBroadcastExec +import org.apache.spark.sql.execution.{SubqueryAdaptiveBroadcastExec, SubqueryBroadcastExec} trait ShimSubqueryBroadcast { @@ -30,4 +30,9 @@ trait ShimSubqueryBroadcast { def getSubqueryBroadcastIndices(sab: SubqueryAdaptiveBroadcastExec): Seq[Int] = { Seq(sab.index) } + + /** Same version shim for SubqueryBroadcastExec. */ + def getSubqueryBroadcastExecIndices(sub: SubqueryBroadcastExec): Seq[Int] = { + Seq(sub.index) + } } diff --git a/spark/src/main/spark-3.5/org/apache/comet/shims/ShimSubqueryBroadcast.scala b/spark/src/main/spark-3.5/org/apache/comet/shims/ShimSubqueryBroadcast.scala index 1ff0935041..292ed2cb18 100644 --- a/spark/src/main/spark-3.5/org/apache/comet/shims/ShimSubqueryBroadcast.scala +++ b/spark/src/main/spark-3.5/org/apache/comet/shims/ShimSubqueryBroadcast.scala @@ -19,7 +19,7 @@ package org.apache.comet.shims -import org.apache.spark.sql.execution.SubqueryAdaptiveBroadcastExec +import org.apache.spark.sql.execution.{SubqueryAdaptiveBroadcastExec, SubqueryBroadcastExec} trait ShimSubqueryBroadcast { @@ -30,4 +30,9 @@ trait ShimSubqueryBroadcast { def getSubqueryBroadcastIndices(sab: SubqueryAdaptiveBroadcastExec): Seq[Int] = { Seq(sab.index) } + + /** Same version shim for SubqueryBroadcastExec. */ + def getSubqueryBroadcastExecIndices(sub: SubqueryBroadcastExec): Seq[Int] = { + Seq(sub.index) + } } diff --git a/spark/src/main/spark-4.0/org/apache/comet/shims/ShimSubqueryBroadcast.scala b/spark/src/main/spark-4.0/org/apache/comet/shims/ShimSubqueryBroadcast.scala index 417dfd46b7..73d9e53c4a 100644 --- a/spark/src/main/spark-4.0/org/apache/comet/shims/ShimSubqueryBroadcast.scala +++ b/spark/src/main/spark-4.0/org/apache/comet/shims/ShimSubqueryBroadcast.scala @@ -19,7 +19,7 @@ package org.apache.comet.shims -import org.apache.spark.sql.execution.SubqueryAdaptiveBroadcastExec +import org.apache.spark.sql.execution.{SubqueryAdaptiveBroadcastExec, SubqueryBroadcastExec} trait ShimSubqueryBroadcast { @@ -30,4 +30,9 @@ trait ShimSubqueryBroadcast { def getSubqueryBroadcastIndices(sab: SubqueryAdaptiveBroadcastExec): Seq[Int] = { sab.indices } + + /** Same version shim for SubqueryBroadcastExec. */ + def getSubqueryBroadcastExecIndices(sub: SubqueryBroadcastExec): Seq[Int] = { + sub.indices + } } diff --git a/spark/src/test/scala/org/apache/comet/exec/CometExecSuite.scala b/spark/src/test/scala/org/apache/comet/exec/CometExecSuite.scala index b3ca0c52ff..16b79e34c2 100644 --- a/spark/src/test/scala/org/apache/comet/exec/CometExecSuite.scala +++ b/spark/src/test/scala/org/apache/comet/exec/CometExecSuite.scala @@ -204,7 +204,9 @@ class CometExecSuite extends CometTestBase { spark.read.parquet(dimPath).createOrReplaceTempView("dpp_dim_bhj") val df = spark.sql( "select * from dpp_fact_bhj join dpp_dim_bhj on fact_date = dim_date where dim_id > 7") - val (_, cometPlan) = checkSparkAnswerAndOperator(df) + // Exclude ReusedExchangeExec — it appears inside the DPP subquery after exchange reuse + val (_, cometPlan) = checkSparkAnswerAndOperator( + df, classOf[ReusedExchangeExec]) val nativeScans = cometPlan.collect { case s: CometNativeScanExec => s } assert(nativeScans.nonEmpty, "Expected CometNativeScanExec in plan") @@ -263,6 +265,77 @@ class CometExecSuite extends CometTestBase { } } + test("DPP broadcast exchange reuse investigation") { + withTempDir { dir => + val path = s"${dir.getAbsolutePath}/data" + withSQLConf(CometConf.COMET_EXEC_ENABLED.key -> "false") { + spark.range(100).selectExpr( + "id % 10 as store_id", "cast(id * 2 as int) as date_id", + "cast(id * 3 as int) as product_id", "cast(id as int) as units_sold") + .write.partitionBy("store_id").parquet(s"$path/fact") + spark.range(10).selectExpr( + "cast(id as int) as store_id", "cast(id as string) as country") + .write.parquet(s"$path/dim") + } + + withSQLConf( + SQLConf.USE_V1_SOURCE_LIST.key -> "parquet", + SQLConf.ADAPTIVE_EXECUTION_ENABLED.key -> "false", + SQLConf.DYNAMIC_PARTITION_PRUNING_REUSE_BROADCAST_ONLY.key -> "true") { + spark.read.parquet(s"$path/fact").createOrReplaceTempView("fact_reuse") + spark.read.parquet(s"$path/dim").createOrReplaceTempView("dim_reuse") + + val df = spark.sql( + """SELECT f.date_id, f.store_id + |FROM fact_reuse f JOIN dim_reuse d + |ON f.store_id = d.store_id + |WHERE d.country = 'DE'""".stripMargin) + df.collect() + val plan = df.queryExecution.executedPlan + // scalastyle:off println + println(s"[REUSE-DEBUG] Plan:\n${plan.treeString}") + + // Walk into subquery expressions to see what's inside + plan.foreach { node => + node.expressions.foreach { expr => + expr.foreach { + case sub: InSubqueryExec => + println(s"[REUSE-DEBUG] Found InSubqueryExec in ${node.getClass.getSimpleName}") + println(s"[REUSE-DEBUG] sub.plan class: ${sub.plan.getClass.getSimpleName}") + println(s"[REUSE-DEBUG] sub.plan tree:\n${sub.plan.treeString}") + sub.plan match { + case sb: SubqueryBroadcastExec => + println(s"[REUSE-DEBUG] SubqueryBroadcast child: " + + s"${sb.child.getClass.getSimpleName}") + println(s"[REUSE-DEBUG] SubqueryBroadcast child tree:\n" + + s"${sb.child.treeString}") + case other => + println(s"[REUSE-DEBUG] sub.plan is: ${other.getClass.getSimpleName}") + } + case _ => + } + } + } + + val reused = collectWithSubqueries(plan) { + case e: ReusedExchangeExec => e + } + println(s"[REUSE-DEBUG] ReusedExchangeExec count: ${reused.size}") + + val broadcasts = collectWithSubqueries(plan) { + case e: BroadcastExchangeExec => ("BroadcastExchangeExec", e: SparkPlan) + case e: CometBroadcastExchangeExec => ("CometBroadcastExchangeExec", e: SparkPlan) + } + println(s"[REUSE-DEBUG] Broadcast exchange count: ${broadcasts.size}") + broadcasts.foreach { case (typ, e) => + println(s"[REUSE-DEBUG] $typ hash=${e.canonicalized.hashCode()}") + println(s"[REUSE-DEBUG] $typ child: ${e.children.map(_.getClass.getSimpleName)}") + } + // scalastyle:on println + } + } + } + test("ShuffleQueryStageExec could be direct child node of CometBroadcastExchangeExec") { withSQLConf(CometConf.COMET_SHUFFLE_MODE.key -> "jvm") { val table = "src" From 0837d9e6549b0300b5b0170c18d57f900c22b4e3 Mon Sep 17 00:00:00 2001 From: Matt Butrovich Date: Tue, 21 Apr 2026 09:12:37 -0400 Subject: [PATCH 09/21] Passes tests with reuse. --- .../comet/CometSubqueryBroadcastExec.scala | 163 ++++++++++++++++++ 1 file changed, 163 insertions(+) create mode 100644 spark/src/main/scala/org/apache/spark/sql/comet/CometSubqueryBroadcastExec.scala diff --git a/spark/src/main/scala/org/apache/spark/sql/comet/CometSubqueryBroadcastExec.scala b/spark/src/main/scala/org/apache/spark/sql/comet/CometSubqueryBroadcastExec.scala new file mode 100644 index 0000000000..359a91c548 --- /dev/null +++ b/spark/src/main/scala/org/apache/spark/sql/comet/CometSubqueryBroadcastExec.scala @@ -0,0 +1,163 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.spark.sql.comet + +import java.util.concurrent.{Future => JFuture} + +import scala.concurrent.ExecutionContext +import scala.concurrent.duration.Duration + +import org.apache.spark.rdd.RDD +import org.apache.spark.sql.catalyst.InternalRow +import org.apache.spark.sql.catalyst.expressions._ +import org.apache.spark.sql.catalyst.plans.QueryPlan +import org.apache.spark.sql.comet.util.Utils +import org.apache.spark.sql.errors.QueryExecutionErrors +import org.apache.spark.sql.execution._ +import org.apache.spark.sql.execution.metric.SQLMetrics +import org.apache.spark.sql.internal.{SQLConf, StaticSQLConf} +import org.apache.spark.util.ThreadUtils +import org.apache.spark.util.io.ChunkedByteBuffer + +/** + * Comet replacement for SubqueryBroadcastExec that consumes Arrow broadcast data from a + * CometBroadcastExchangeExec instead of HashedRelation from BroadcastExchangeExec. + * + * This enables broadcast exchange reuse between DPP subqueries and broadcast hash joins + * when CometExecRule converts BroadcastExchangeExec to CometBroadcastExchangeExec. + * Without this, the two exchanges have different types and canonical forms, so Spark's + * ReuseExchangeAndSubquery (which runs after Comet rules) cannot match them. + * + * @param indices the indices of the join keys in the list of keys from the build side + * @param buildKeys the join keys from the build side of the join + * @param child the CometBroadcastExchangeExec (or ReusedExchangeExec after reuse) + */ +case class CometSubqueryBroadcastExec( + name: String, + indices: Seq[Int], + buildKeys: Seq[Expression], + child: SparkPlan) + extends BaseSubqueryExec + with UnaryExecNode { + + override def output: Seq[Attribute] = { + indices.map { idx => + val key = buildKeys(idx) + val attrName = key match { + case n: NamedExpression => n.name + case Cast(n: NamedExpression, _, _, _) => n.name + case _ => s"key_$idx" + } + AttributeReference(attrName, key.dataType, key.nullable)() + } + } + + override lazy val metrics = Map( + "numOutputRows" -> SQLMetrics.createMetric(sparkContext, "number of output rows"), + "dataSize" -> SQLMetrics.createMetric(sparkContext, "data size (bytes)"), + "collectTime" -> SQLMetrics.createMetric(sparkContext, "time to collect (ms)")) + + override def doCanonicalize(): SparkPlan = { + val keys = buildKeys.map(k => QueryPlan.normalizeExpressions(k, child.output)) + CometSubqueryBroadcastExec("dpp", indices, keys, child.canonicalized) + } + + @transient + private lazy val relationFuture: JFuture[Array[InternalRow]] = { + val executionId = sparkContext.getLocalProperty(SQLExecution.EXECUTION_ID_KEY) + SQLExecution.withThreadLocalCaptured[Array[InternalRow]]( + session, + CometSubqueryBroadcastExec.executionContext) { + SQLExecution.withExecutionId(session, executionId) { + val beforeCollect = System.nanoTime() + + // Get the Arrow broadcast from CometBroadcastExchangeExec + val broadcasted = child.executeBroadcast[Array[ChunkedByteBuffer]]() + val arrowBatches = broadcasted.value + + // Decode Arrow batches and extract key column values + val keyIndices = indices.map { idx => + val key = buildKeys(idx) + // Find the column index in the broadcast output that matches the build key + key match { + case attr: Attribute => + child.output.indexWhere(_.exprId == attr.exprId) + case Cast(attr: Attribute, _, _, _) => + child.output.indexWhere(_.exprId == attr.exprId) + case _ => idx + } + } + + val rows = arrowBatches.iterator + .flatMap(Utils.decodeBatches(_, this.getClass.getSimpleName)) + .flatMap { batch => + val numRows = batch.numRows() + (0 until numRows).iterator.map { rowIdx => + val row = batch.getRow(rowIdx) + val projected = new GenericInternalRow(keyIndices.length) + keyIndices.zipWithIndex.foreach { case (colIdx, outIdx) => + projected.update(outIdx, row.get(colIdx, output(outIdx).dataType)) + } + projected.asInstanceOf[InternalRow].copy() + } + } + .toArray + .distinct + + val beforeBuild = System.nanoTime() + longMetric("collectTime") += (beforeBuild - beforeCollect) / 1000000 + longMetric("numOutputRows") += rows.length + // Convert to UnsafeRow for consistent size metric and to match SubqueryBroadcastExec + val unsafeProj = UnsafeProjection.create(output.map(_.dataType).toArray) + val unsafeRows = rows.map(r => unsafeProj(r).copy()) + val dataSize = unsafeRows.map(_.getSizeInBytes.toLong).sum + longMetric("dataSize") += dataSize + SQLMetrics.postDriverMetricUpdates(sparkContext, executionId, metrics.values.toSeq) + + unsafeRows.asInstanceOf[Array[InternalRow]] + } + } + } + + protected override def doPrepare(): Unit = { + relationFuture + } + + protected override def doExecute(): RDD[InternalRow] = { + throw QueryExecutionErrors.executeCodePathUnsupportedError("CometSubqueryBroadcastExec") + } + + override def executeCollect(): Array[InternalRow] = { + ThreadUtils.awaitResult(relationFuture, Duration.Inf) + } + + override def stringArgs: Iterator[Any] = super.stringArgs ++ Iterator(s"[id=#$id]") + + override protected def withNewChildInternal( + newChild: SparkPlan): CometSubqueryBroadcastExec = + copy(child = newChild) +} + +object CometSubqueryBroadcastExec { + private[comet] val executionContext = ExecutionContext.fromExecutorService( + ThreadUtils.newDaemonCachedThreadPool( + "comet-dynamicpruning", + SQLConf.get.getConf(StaticSQLConf.BROADCAST_EXCHANGE_MAX_THREAD_THRESHOLD))) +} From 416e5b8d7df3db09303dd177344f04e3eb705e7b Mon Sep 17 00:00:00 2001 From: Matt Butrovich Date: Tue, 21 Apr 2026 09:22:20 -0400 Subject: [PATCH 10/21] Convert prints to logs. --- .../apache/comet/rules/CometExecRule.scala | 116 ++++++------------ .../spark/sql/comet/CometNativeScanExec.scala | 30 +---- .../comet/CometSubqueryBroadcastExec.scala | 20 +-- .../apache/comet/exec/CometExecSuite.scala | 87 ++++++------- 4 files changed, 93 insertions(+), 160 deletions(-) diff --git a/spark/src/main/scala/org/apache/comet/rules/CometExecRule.scala b/spark/src/main/scala/org/apache/comet/rules/CometExecRule.scala index 8d87c9765b..0da76475bc 100644 --- a/spark/src/main/scala/org/apache/comet/rules/CometExecRule.scala +++ b/spark/src/main/scala/org/apache/comet/rules/CometExecRule.scala @@ -301,42 +301,7 @@ case class CometExecRule(session: SparkSession) } } - // scalastyle:off println plan.transformUp { case op => - val hasSubqueryExpr = op.expressions.exists(_.exists { - case _: InSubqueryExec => true - case _ => false - }) - if (hasSubqueryExpr) { - println(s"[RULE-DEBUG] convertNode on ${op.getClass.getSimpleName} " + - s"which HAS InSubqueryExec expressions") - op.expressions.foreach { expr => - expr.foreach { - case sub: InSubqueryExec => - println(s"[RULE-DEBUG] InSubqueryExec.plan: ${sub.plan.getClass.getSimpleName}") - sub.plan match { - case sb: SubqueryBroadcastExec => - println(s"[RULE-DEBUG] SubqueryBroadcast.child: " + - s"${sb.child.getClass.getSimpleName}") - sb.child match { - case b: BroadcastExchangeExec => - println(s"[RULE-DEBUG] BroadcastExchange.child: " + - s"${b.child.getClass.getSimpleName}") - println(s"[RULE-DEBUG] BroadcastExchange.child is CometNative? " + - s"${b.child.isInstanceOf[CometNativeExec]}") - println(s"[RULE-DEBUG] BroadcastExchange.children all CometNative? " + - s"${b.children.forall(_.isInstanceOf[CometNativeExec])}") - case other => - println(s"[RULE-DEBUG] SubqueryBroadcast.child is: " + - s"${other.getClass.getSimpleName}") - } - case other => - println(s"[RULE-DEBUG] sub.plan is: ${other.getClass.getSimpleName}") - } - case _ => - } - } - } val converted = convertNode(op) // Replace SubqueryBroadcastExec with CometSubqueryBroadcastExec in DPP expressions // when the broadcast child has a Comet plan underneath. This enables exchange reuse @@ -344,57 +309,52 @@ case class CometExecRule(session: SparkSession) // will have the same CometBroadcastExchangeExec type and canonical form. convertSubqueryBroadcasts(converted) } - // scalastyle:on println } /** * Replace SubqueryBroadcastExec with CometSubqueryBroadcastExec in a node's expressions. * - * When CometExecRule converts BroadcastExchangeExec to CometBroadcastExchangeExec on the - * join side, the DPP subquery still references the original BroadcastExchangeExec. - * ReuseExchangeAndSubquery (which runs after Comet rules) can't match them because they - * have different types. By replacing SubqueryBroadcastExec with CometSubqueryBroadcastExec - * (which wraps a CometBroadcastExchangeExec), both sides have the same exchange type and - * reuse works. + * When CometExecRule converts BroadcastExchangeExec to CometBroadcastExchangeExec on the join + * side, the DPP subquery still references the original BroadcastExchangeExec. + * ReuseExchangeAndSubquery (which runs after Comet rules) can't match them because they have + * different types. By replacing SubqueryBroadcastExec with CometSubqueryBroadcastExec (which + * wraps a CometBroadcastExchangeExec), both sides have the same exchange type and reuse works. * - * The BroadcastExchangeExec in the subquery has a CometNativeColumnarToRowExec child - * (inserted by ApplyColumnarRulesAndInsertTransitions because BroadcastExchangeExec expects - * row input). We strip this transition and create CometBroadcastExchangeExec with the - * underlying Comet plan directly. + * The BroadcastExchangeExec in the subquery has a CometNativeColumnarToRowExec child (inserted + * by ApplyColumnarRulesAndInsertTransitions because BroadcastExchangeExec expects row input). + * We strip this transition and create CometBroadcastExchangeExec with the underlying Comet plan + * directly. */ private def convertSubqueryBroadcasts(plan: SparkPlan): SparkPlan = { - plan.transformExpressionsUp { - case inSub: InSubqueryExec => - inSub.plan match { - case sub: SubqueryBroadcastExec => - sub.child match { - case b: BroadcastExchangeExec => - // The BroadcastExchangeExec child is CometNativeColumnarToRowExec wrapping - // a Comet plan. Strip the row transition to get the columnar Comet plan. - val cometChild = b.child match { - case c2r: CometNativeColumnarToRowExec => c2r.child - case other => other - } - if (cometChild.isInstanceOf[CometNativeExec]) { - // scalastyle:off println - println(s"[RULE-DEBUG] Converting SubqueryBroadcastExec to " + - s"CometSubqueryBroadcastExec, cometChild=${cometChild.getClass.getSimpleName}") - // scalastyle:on println - val cometBroadcast = CometBroadcastExchangeExec( - b, b.output, b.mode, cometChild) - val cometSub = CometSubqueryBroadcastExec( - sub.name, - getSubqueryBroadcastExecIndices(sub), - sub.buildKeys, - cometBroadcast) - inSub.withNewPlan(cometSub) - } else { - inSub - } - case _ => inSub - } - case _ => inSub - } + plan.transformExpressionsUp { case inSub: InSubqueryExec => + inSub.plan match { + case sub: SubqueryBroadcastExec => + sub.child match { + case b: BroadcastExchangeExec => + // The BroadcastExchangeExec child is CometNativeColumnarToRowExec wrapping + // a Comet plan. Strip the row transition to get the columnar Comet plan. + val cometChild = b.child match { + case c2r: CometNativeColumnarToRowExec => c2r.child + case other => other + } + if (cometChild.isInstanceOf[CometNativeExec]) { + logInfo( + s"Converting SubqueryBroadcastExec to " + + s"CometSubqueryBroadcastExec for DPP exchange reuse") + val cometBroadcast = CometBroadcastExchangeExec(b, b.output, b.mode, cometChild) + val cometSub = CometSubqueryBroadcastExec( + sub.name, + getSubqueryBroadcastExecIndices(sub), + sub.buildKeys, + cometBroadcast) + inSub.withNewPlan(cometSub) + } else { + inSub + } + case _ => inSub + } + case _ => inSub + } } } diff --git a/spark/src/main/scala/org/apache/spark/sql/comet/CometNativeScanExec.scala b/spark/src/main/scala/org/apache/spark/sql/comet/CometNativeScanExec.scala index 05219512d2..aaa3e8c104 100644 --- a/spark/src/main/scala/org/apache/spark/sql/comet/CometNativeScanExec.scala +++ b/spark/src/main/scala/org/apache/spark/sql/comet/CometNativeScanExec.scala @@ -176,37 +176,20 @@ case class CometNativeScanExec( * partition's files (lazily, as tasks are scheduled). */ @transient private lazy val serializedPartitionData: (Array[Byte], Array[Array[Byte]]) = { - // scalastyle:off println - println(s"[DPP-DEBUG] serializedPartitionData: checking partitionFilters") - partitionFilters.foreach { - case DynamicPruningExpression(e: InSubqueryExec) => - println(s"[DPP-DEBUG] InSubqueryExec plan=${e.plan.getClass.getSimpleName} " + - s"values empty=${e.values().isEmpty}") - case other => - println(s"[DPP-DEBUG] filter: ${other.getClass.getSimpleName}") - } - // scalastyle:on println // Ensure DPP subqueries are resolved before accessing file partitions. // serializedPartitionData can be triggered from findAllPlanData (via commonData) on a // different execution path than the standard prepare() -> executeSubqueries() flow // (e.g., from a BroadcastExchangeExec thread). We must resolve DPP here explicitly. partitionFilters.foreach { case DynamicPruningExpression(e: InSubqueryExec) if e.values().isEmpty => - // scalastyle:off println - println(s"[DPP-DEBUG] calling updateResult on InSubqueryExec " + - s"plan=${e.plan.getClass.getSimpleName} id=${System.identityHashCode(e)}") - // scalastyle:on println + logDebug(s"Resolving DPP subquery: plan=${e.plan.getClass.getSimpleName}") try { e.updateResult() - // scalastyle:off println - println(s"[DPP-DEBUG] updateResult succeeded, values empty=${e.values().isEmpty}") - // scalastyle:on println + logDebug(s"DPP subquery resolved successfully") } catch { - // scalastyle:off println case ex: Exception => - println(s"[DPP-DEBUG] updateResult FAILED: ${ex.getMessage}") + logError(s"DPP subquery resolution failed: ${ex.getMessage}") throw ex - // scalastyle:on println } case _ => } @@ -216,10 +199,9 @@ case class CometNativeScanExec( if (scan != null) { scan.partitionFilters.foreach { case DynamicPruningExpression(e: InSubqueryExec) if e.values().isEmpty => - // scalastyle:off println - println(s"[DPP-DEBUG] also resolving scan's InSubqueryExec " + - s"plan=${e.plan.getClass.getSimpleName} id=${System.identityHashCode(e)}") - // scalastyle:on println + logDebug( + s"Resolving CometScanExec DPP subquery: " + + s"plan=${e.plan.getClass.getSimpleName}") e.updateResult() case _ => } diff --git a/spark/src/main/scala/org/apache/spark/sql/comet/CometSubqueryBroadcastExec.scala b/spark/src/main/scala/org/apache/spark/sql/comet/CometSubqueryBroadcastExec.scala index 359a91c548..b184eea7d6 100644 --- a/spark/src/main/scala/org/apache/spark/sql/comet/CometSubqueryBroadcastExec.scala +++ b/spark/src/main/scala/org/apache/spark/sql/comet/CometSubqueryBroadcastExec.scala @@ -40,14 +40,17 @@ import org.apache.spark.util.io.ChunkedByteBuffer * Comet replacement for SubqueryBroadcastExec that consumes Arrow broadcast data from a * CometBroadcastExchangeExec instead of HashedRelation from BroadcastExchangeExec. * - * This enables broadcast exchange reuse between DPP subqueries and broadcast hash joins - * when CometExecRule converts BroadcastExchangeExec to CometBroadcastExchangeExec. - * Without this, the two exchanges have different types and canonical forms, so Spark's - * ReuseExchangeAndSubquery (which runs after Comet rules) cannot match them. + * This enables broadcast exchange reuse between DPP subqueries and broadcast hash joins when + * CometExecRule converts BroadcastExchangeExec to CometBroadcastExchangeExec. Without this, the + * two exchanges have different types and canonical forms, so Spark's ReuseExchangeAndSubquery + * (which runs after Comet rules) cannot match them. * - * @param indices the indices of the join keys in the list of keys from the build side - * @param buildKeys the join keys from the build side of the join - * @param child the CometBroadcastExchangeExec (or ReusedExchangeExec after reuse) + * @param indices + * the indices of the join keys in the list of keys from the build side + * @param buildKeys + * the join keys from the build side of the join + * @param child + * the CometBroadcastExchangeExec (or ReusedExchangeExec after reuse) */ case class CometSubqueryBroadcastExec( name: String, @@ -150,8 +153,7 @@ case class CometSubqueryBroadcastExec( override def stringArgs: Iterator[Any] = super.stringArgs ++ Iterator(s"[id=#$id]") - override protected def withNewChildInternal( - newChild: SparkPlan): CometSubqueryBroadcastExec = + override protected def withNewChildInternal(newChild: SparkPlan): CometSubqueryBroadcastExec = copy(child = newChild) } diff --git a/spark/src/test/scala/org/apache/comet/exec/CometExecSuite.scala b/spark/src/test/scala/org/apache/comet/exec/CometExecSuite.scala index 16b79e34c2..08c72cf5c7 100644 --- a/spark/src/test/scala/org/apache/comet/exec/CometExecSuite.scala +++ b/spark/src/test/scala/org/apache/comet/exec/CometExecSuite.scala @@ -205,8 +205,7 @@ class CometExecSuite extends CometTestBase { val df = spark.sql( "select * from dpp_fact_bhj join dpp_dim_bhj on fact_date = dim_date where dim_id > 7") // Exclude ReusedExchangeExec — it appears inside the DPP subquery after exchange reuse - val (_, cometPlan) = checkSparkAnswerAndOperator( - df, classOf[ReusedExchangeExec]) + val (_, cometPlan) = checkSparkAnswerAndOperator(df, classOf[ReusedExchangeExec]) val nativeScans = cometPlan.collect { case s: CometNativeScanExec => s } assert(nativeScans.nonEmpty, "Expected CometNativeScanExec in plan") @@ -265,17 +264,25 @@ class CometExecSuite extends CometTestBase { } } - test("DPP broadcast exchange reuse investigation") { + test("non-AQE DPP with BHJ reuses broadcast exchange") { withTempDir { dir => val path = s"${dir.getAbsolutePath}/data" withSQLConf(CometConf.COMET_EXEC_ENABLED.key -> "false") { - spark.range(100).selectExpr( - "id % 10 as store_id", "cast(id * 2 as int) as date_id", - "cast(id * 3 as int) as product_id", "cast(id as int) as units_sold") - .write.partitionBy("store_id").parquet(s"$path/fact") - spark.range(10).selectExpr( - "cast(id as int) as store_id", "cast(id as string) as country") - .write.parquet(s"$path/dim") + spark + .range(100) + .selectExpr( + "id % 10 as store_id", + "cast(id * 2 as int) as date_id", + "cast(id * 3 as int) as product_id", + "cast(id as int) as units_sold") + .write + .partitionBy("store_id") + .parquet(s"$path/fact") + spark + .range(10) + .selectExpr("cast(id as int) as store_id", "cast(id as string) as country") + .write + .parquet(s"$path/dim") } withSQLConf( @@ -285,53 +292,35 @@ class CometExecSuite extends CometTestBase { spark.read.parquet(s"$path/fact").createOrReplaceTempView("fact_reuse") spark.read.parquet(s"$path/dim").createOrReplaceTempView("dim_reuse") - val df = spark.sql( - """SELECT f.date_id, f.store_id + val df = spark.sql("""SELECT f.date_id, f.store_id |FROM fact_reuse f JOIN dim_reuse d |ON f.store_id = d.store_id |WHERE d.country = 'DE'""".stripMargin) - df.collect() - val plan = df.queryExecution.executedPlan - // scalastyle:off println - println(s"[REUSE-DEBUG] Plan:\n${plan.treeString}") - - // Walk into subquery expressions to see what's inside - plan.foreach { node => - node.expressions.foreach { expr => - expr.foreach { - case sub: InSubqueryExec => - println(s"[REUSE-DEBUG] Found InSubqueryExec in ${node.getClass.getSimpleName}") - println(s"[REUSE-DEBUG] sub.plan class: ${sub.plan.getClass.getSimpleName}") - println(s"[REUSE-DEBUG] sub.plan tree:\n${sub.plan.treeString}") - sub.plan match { - case sb: SubqueryBroadcastExec => - println(s"[REUSE-DEBUG] SubqueryBroadcast child: " + - s"${sb.child.getClass.getSimpleName}") - println(s"[REUSE-DEBUG] SubqueryBroadcast child tree:\n" + - s"${sb.child.treeString}") - case other => - println(s"[REUSE-DEBUG] sub.plan is: ${other.getClass.getSimpleName}") - } - case _ => - } - } - } + val (_, cometPlan) = checkSparkAnswer(df) - val reused = collectWithSubqueries(plan) { - case e: ReusedExchangeExec => e + // DPP subquery should use CometSubqueryBroadcastExec (not SubqueryBroadcastExec) + val cometSubqueries = collectWithSubqueries(cometPlan) { + case s: CometSubqueryBroadcastExec => s } - println(s"[REUSE-DEBUG] ReusedExchangeExec count: ${reused.size}") + assert( + cometSubqueries.nonEmpty, + "Expected CometSubqueryBroadcastExec in plan for exchange reuse") - val broadcasts = collectWithSubqueries(plan) { - case e: BroadcastExchangeExec => ("BroadcastExchangeExec", e: SparkPlan) - case e: CometBroadcastExchangeExec => ("CometBroadcastExchangeExec", e: SparkPlan) + // Broadcast exchange should be reused — only one CometBroadcastExchangeExec, + // the other replaced by ReusedExchangeExec + val reused = collectWithSubqueries(cometPlan) { case e: ReusedExchangeExec => + e } - println(s"[REUSE-DEBUG] Broadcast exchange count: ${broadcasts.size}") - broadcasts.foreach { case (typ, e) => - println(s"[REUSE-DEBUG] $typ hash=${e.canonicalized.hashCode()}") - println(s"[REUSE-DEBUG] $typ child: ${e.children.map(_.getClass.getSimpleName)}") + assert( + reused.nonEmpty, + s"Expected ReusedExchangeExec for broadcast exchange reuse:\n${cometPlan.treeString}") + + val broadcasts = collectWithSubqueries(cometPlan) { case e: CometBroadcastExchangeExec => + e } - // scalastyle:on println + assert( + broadcasts.size == 1, + s"Expected exactly 1 CometBroadcastExchangeExec (other reused):\n${cometPlan.treeString}") } } } From 4cb98f941acecf9c35db9f9bb25ccfd60523198c Mon Sep 17 00:00:00 2001 From: Matt Butrovich Date: Tue, 21 Apr 2026 09:51:57 -0400 Subject: [PATCH 11/21] Update spark diffs, new golden files. --- dev/diffs/3.4.3.diff | 40 +- dev/diffs/3.5.8.diff | 43 +- dev/diffs/4.0.1.diff | 44 +- .../apache/comet/rules/CometScanRule.scala | 10 +- .../q1.native_datafusion/extended.txt | 24 +- .../q10.native_datafusion/extended.txt | 35 +- .../q11.native_datafusion/extended.txt | 38 +- .../q12.native_datafusion/extended.txt | 13 +- .../q13.native_datafusion/extended.txt | 13 +- .../q14a.native_datafusion/extended.txt | 266 ++-- .../q14b.native_datafusion/extended.txt | 221 ++- .../q15.native_datafusion/extended.txt | 13 +- .../q17.native_datafusion/extended.txt | 35 +- .../q18.native_datafusion/extended.txt | 13 +- .../q20.native_datafusion/extended.txt | 13 +- .../q21.native_datafusion/extended.txt | 11 +- .../q22.native_datafusion/extended.txt | 13 +- .../q23a.native_datafusion/extended.txt | 57 +- .../q23b.native_datafusion/extended.txt | 57 +- .../q25.native_datafusion/extended.txt | 35 +- .../q26.native_datafusion/extended.txt | 13 +- .../q27.native_datafusion/extended.txt | 13 +- .../q29.native_datafusion/extended.txt | 35 +- .../q30.native_datafusion/extended.txt | 24 +- .../q31.native_datafusion/extended.txt | 56 +- .../q32.native_datafusion/extended.txt | 24 +- .../q33.native_datafusion/extended.txt | 35 +- .../q34.native_datafusion/extended.txt | 13 +- .../q35.native_datafusion/extended.txt | 35 +- .../q36.native_datafusion/extended.txt | 13 +- .../q37.native_datafusion/extended.txt | 13 +- .../q38.native_datafusion/extended.txt | 35 +- .../q39a.native_datafusion/extended.txt | 24 +- .../q39b.native_datafusion/extended.txt | 24 +- .../q4.native_datafusion/extended.txt | 56 +- .../q40.native_datafusion/extended.txt | 11 +- .../q45.native_datafusion/extended.txt | 13 +- .../q46.native_datafusion/extended.txt | 13 +- .../q47.native_datafusion/extended.txt | 29 +- .../q48.native_datafusion/extended.txt | 13 +- .../q49.native_datafusion/extended.txt | 35 +- .../q5.native_datafusion/extended.txt | 68 +- .../q50.native_datafusion/extended.txt | 13 +- .../q51.native_datafusion/extended.txt | 24 +- .../q53.native_datafusion/extended.txt | 13 +- .../q54.native_datafusion/extended.txt | 85 +- .../q56.native_datafusion/extended.txt | 35 +- .../q57.native_datafusion/extended.txt | 29 +- .../q58.native_datafusion/extended.txt | 95 +- .../q6.native_datafusion/extended.txt | 38 +- .../q60.native_datafusion/extended.txt | 35 +- .../q61.native_datafusion/extended.txt | 24 +- .../q63.native_datafusion/extended.txt | 13 +- .../q64.native_datafusion/extended.txt | 20 +- .../q65.native_datafusion/extended.txt | 24 +- .../q66.native_datafusion/extended.txt | 20 +- .../q67.native_datafusion/extended.txt | 13 +- .../q68.native_datafusion/extended.txt | 13 +- .../q69.native_datafusion/extended.txt | 35 +- .../q7.native_datafusion/extended.txt | 13 +- .../q70.native_datafusion/extended.txt | 24 +- .../q71.native_datafusion/extended.txt | 35 +- .../q72.native_datafusion/extended.txt | 13 +- .../q73.native_datafusion/extended.txt | 13 +- .../q74.native_datafusion/extended.txt | 38 +- .../q75.native_datafusion/extended.txt | 56 +- .../q77.native_datafusion/extended.txt | 68 +- .../q78.native_datafusion/extended.txt | 29 +- .../q79.native_datafusion/extended.txt | 13 +- .../q8.native_datafusion/extended.txt | 13 +- .../q80.native_datafusion/extended.txt | 35 +- .../q81.native_datafusion/extended.txt | 24 +- .../q82.native_datafusion/extended.txt | 13 +- .../q83.native_datafusion/extended.txt | 89 +- .../q85.native_datafusion/extended.txt | 13 +- .../q86.native_datafusion/extended.txt | 13 +- .../q87.native_datafusion/extended.txt | 35 +- .../q89.native_datafusion/extended.txt | 13 +- .../q91.native_datafusion/extended.txt | 13 +- .../q92.native_datafusion/extended.txt | 24 +- .../q97.native_datafusion/extended.txt | 24 +- .../q98.native_datafusion/extended.txt | 13 +- .../q1.native_datafusion/extended.txt | 24 +- .../q10.native_datafusion/extended.txt | 35 +- .../q11.native_datafusion/extended.txt | 38 +- .../q12.native_datafusion/extended.txt | 13 +- .../q13.native_datafusion/extended.txt | 13 +- .../q14a.native_datafusion/extended.txt | 266 ++-- .../q14b.native_datafusion/extended.txt | 229 ++- .../q15.native_datafusion/extended.txt | 13 +- .../q17.native_datafusion/extended.txt | 35 +- .../q18.native_datafusion/extended.txt | 13 +- .../q20.native_datafusion/extended.txt | 13 +- .../q21.native_datafusion/extended.txt | 11 +- .../q22.native_datafusion/extended.txt | 13 +- .../q23a.native_datafusion/extended.txt | 57 +- .../q23b.native_datafusion/extended.txt | 57 +- .../q25.native_datafusion/extended.txt | 35 +- .../q26.native_datafusion/extended.txt | 13 +- .../q27.native_datafusion/extended.txt | 13 +- .../q29.native_datafusion/extended.txt | 35 +- .../q30.native_datafusion/extended.txt | 24 +- .../q31.native_datafusion/extended.txt | 56 +- .../q32.native_datafusion/extended.txt | 24 +- .../q33.native_datafusion/extended.txt | 35 +- .../q34.native_datafusion/extended.txt | 13 +- .../q35.native_datafusion/extended.txt | 35 +- .../q36.native_datafusion/extended.txt | 13 +- .../q37.native_datafusion/extended.txt | 13 +- .../q38.native_datafusion/extended.txt | 35 +- .../q39a.native_datafusion/extended.txt | 24 +- .../q39b.native_datafusion/extended.txt | 24 +- .../q4.native_datafusion/extended.txt | 56 +- .../q40.native_datafusion/extended.txt | 11 +- .../q45.native_datafusion/extended.txt | 13 +- .../q46.native_datafusion/extended.txt | 13 +- .../q47.native_datafusion/extended.txt | 29 +- .../q48.native_datafusion/extended.txt | 13 +- .../q49.native_datafusion/extended.txt | 35 +- .../q5.native_datafusion/extended.txt | 68 +- .../q50.native_datafusion/extended.txt | 13 +- .../q51.native_datafusion/extended.txt | 24 +- .../q53.native_datafusion/extended.txt | 13 +- .../q54.native_datafusion/extended.txt | 99 +- .../q56.native_datafusion/extended.txt | 35 +- .../q57.native_datafusion/extended.txt | 29 +- .../q58.native_datafusion/extended.txt | 125 +- .../q6.native_datafusion/extended.txt | 45 +- .../q60.native_datafusion/extended.txt | 35 +- .../q61.native_datafusion/extended.txt | 24 +- .../q63.native_datafusion/extended.txt | 13 +- .../q64.native_datafusion/extended.txt | 20 +- .../q65.native_datafusion/extended.txt | 24 +- .../q66.native_datafusion/extended.txt | 20 +- .../q67.native_datafusion/extended.txt | 13 +- .../q68.native_datafusion/extended.txt | 13 +- .../q69.native_datafusion/extended.txt | 35 +- .../q7.native_datafusion/extended.txt | 13 +- .../q70.native_datafusion/extended.txt | 24 +- .../q71.native_datafusion/extended.txt | 35 +- .../q72.native_datafusion/extended.txt | 13 +- .../q73.native_datafusion/extended.txt | 13 +- .../q74.native_datafusion/extended.txt | 38 +- .../q75.native_datafusion/extended.txt | 56 +- .../q77.native_datafusion/extended.txt | 68 +- .../q78.native_datafusion/extended.txt | 29 +- .../q79.native_datafusion/extended.txt | 13 +- .../q8.native_datafusion/extended.txt | 13 +- .../q80.native_datafusion/extended.txt | 35 +- .../q81.native_datafusion/extended.txt | 24 +- .../q82.native_datafusion/extended.txt | 13 +- .../q83.ansi.native_datafusion/extended.txt | 89 +- .../q85.native_datafusion/extended.txt | 13 +- .../q86.native_datafusion/extended.txt | 13 +- .../q87.native_datafusion/extended.txt | 35 +- .../q89.native_datafusion/extended.txt | 13 +- .../q91.native_datafusion/extended.txt | 13 +- .../q92.native_datafusion/extended.txt | 24 +- .../q97.native_datafusion/extended.txt | 24 +- .../q98.native_datafusion/extended.txt | 13 +- .../q1.native_datafusion/extended.txt | 24 +- .../q10.native_datafusion/extended.txt | 35 +- .../q11.native_datafusion/extended.txt | 38 +- .../q12.native_datafusion/extended.txt | 13 +- .../q13.native_datafusion/extended.txt | 13 +- .../q14a.native_datafusion/extended.txt | 266 ++-- .../q14b.native_datafusion/extended.txt | 221 ++- .../q15.native_datafusion/extended.txt | 13 +- .../q17.native_datafusion/extended.txt | 35 +- .../q18.native_datafusion/extended.txt | 13 +- .../q20.native_datafusion/extended.txt | 13 +- .../q21.native_datafusion/extended.txt | 11 +- .../q22.native_datafusion/extended.txt | 13 +- .../q23a.native_datafusion/extended.txt | 57 +- .../q23b.native_datafusion/extended.txt | 57 +- .../q25.native_datafusion/extended.txt | 35 +- .../q26.native_datafusion/extended.txt | 13 +- .../q27.native_datafusion/extended.txt | 13 +- .../q29.native_datafusion/extended.txt | 35 +- .../q30.native_datafusion/extended.txt | 24 +- .../q31.native_datafusion/extended.txt | 56 +- .../q32.native_datafusion/extended.txt | 24 +- .../q33.native_datafusion/extended.txt | 35 +- .../q34.native_datafusion/extended.txt | 13 +- .../q35.native_datafusion/extended.txt | 35 +- .../q36.native_datafusion/extended.txt | 13 +- .../q37.native_datafusion/extended.txt | 13 +- .../q38.native_datafusion/extended.txt | 35 +- .../q39a.native_datafusion/extended.txt | 24 +- .../q39b.native_datafusion/extended.txt | 24 +- .../q4.native_datafusion/extended.txt | 56 +- .../q40.native_datafusion/extended.txt | 11 +- .../q45.native_datafusion/extended.txt | 13 +- .../q46.native_datafusion/extended.txt | 13 +- .../q47.native_datafusion/extended.txt | 29 +- .../q48.native_datafusion/extended.txt | 13 +- .../q49.native_datafusion/extended.txt | 35 +- .../q5.native_datafusion/extended.txt | 68 +- .../q50.native_datafusion/extended.txt | 13 +- .../q51.native_datafusion/extended.txt | 24 +- .../q53.native_datafusion/extended.txt | 13 +- .../q54.native_datafusion/extended.txt | 85 +- .../q56.native_datafusion/extended.txt | 35 +- .../q57.native_datafusion/extended.txt | 29 +- .../q58.native_datafusion/extended.txt | 95 +- .../q6.native_datafusion/extended.txt | 38 +- .../q60.native_datafusion/extended.txt | 35 +- .../q61.native_datafusion/extended.txt | 24 +- .../q63.native_datafusion/extended.txt | 13 +- .../q64.native_datafusion/extended.txt | 20 +- .../q65.native_datafusion/extended.txt | 24 +- .../q66.native_datafusion/extended.txt | 20 +- .../q67.native_datafusion/extended.txt | 13 +- .../q68.native_datafusion/extended.txt | 13 +- .../q69.native_datafusion/extended.txt | 35 +- .../q7.native_datafusion/extended.txt | 13 +- .../q70.native_datafusion/extended.txt | 24 +- .../q71.native_datafusion/extended.txt | 35 +- .../q72.native_datafusion/extended.txt | 13 +- .../q73.native_datafusion/extended.txt | 13 +- .../q74.native_datafusion/extended.txt | 38 +- .../q75.native_datafusion/extended.txt | 56 +- .../q77.native_datafusion/extended.txt | 68 +- .../q78.native_datafusion/extended.txt | 29 +- .../q79.native_datafusion/extended.txt | 13 +- .../q8.native_datafusion/extended.txt | 13 +- .../q80.native_datafusion/extended.txt | 35 +- .../q81.native_datafusion/extended.txt | 24 +- .../q82.native_datafusion/extended.txt | 13 +- .../q83.native_datafusion/extended.txt | 89 +- .../q85.native_datafusion/extended.txt | 13 +- .../q86.native_datafusion/extended.txt | 13 +- .../q87.native_datafusion/extended.txt | 35 +- .../q89.native_datafusion/extended.txt | 13 +- .../q91.native_datafusion/extended.txt | 13 +- .../q92.native_datafusion/extended.txt | 24 +- .../q97.native_datafusion/extended.txt | 24 +- .../q98.native_datafusion/extended.txt | 13 +- .../q10a.native_datafusion/extended.txt | 35 +- .../q11.native_datafusion/extended.txt | 38 +- .../q12.native_datafusion/extended.txt | 13 +- .../q14.native_datafusion/extended.txt | 221 ++- .../q14a.native_datafusion/extended.txt | 1322 ++++++++--------- .../q18a.native_datafusion/extended.txt | 57 +- .../q20.native_datafusion/extended.txt | 13 +- .../q22.native_datafusion/extended.txt | 13 +- .../q22a.native_datafusion/extended.txt | 57 +- .../q27a.native_datafusion/extended.txt | 35 +- .../q34.native_datafusion/extended.txt | 13 +- .../q35.native_datafusion/extended.txt | 35 +- .../q35a.native_datafusion/extended.txt | 35 +- .../q36a.native_datafusion/extended.txt | 35 +- .../q47.native_datafusion/extended.txt | 29 +- .../q49.native_datafusion/extended.txt | 35 +- .../q51a.native_datafusion/extended.txt | 90 +- .../q57.native_datafusion/extended.txt | 29 +- .../q5a.native_datafusion/extended.txt | 200 ++- .../q6.native_datafusion/extended.txt | 38 +- .../q64.native_datafusion/extended.txt | 20 +- .../q67a.native_datafusion/extended.txt | 101 +- .../q70a.native_datafusion/extended.txt | 68 +- .../q72.native_datafusion/extended.txt | 13 +- .../q74.native_datafusion/extended.txt | 38 +- .../q75.native_datafusion/extended.txt | 56 +- .../q77a.native_datafusion/extended.txt | 200 ++- .../q78.native_datafusion/extended.txt | 29 +- .../q80a.native_datafusion/extended.txt | 101 +- .../q86a.native_datafusion/extended.txt | 35 +- .../q98.native_datafusion/extended.txt | 13 +- .../q10a.native_datafusion/extended.txt | 35 +- .../q11.native_datafusion/extended.txt | 38 +- .../q12.native_datafusion/extended.txt | 13 +- .../q14.native_datafusion/extended.txt | 229 ++- .../q14a.native_datafusion/extended.txt | 1322 ++++++++--------- .../q18a.native_datafusion/extended.txt | 57 +- .../q20.native_datafusion/extended.txt | 13 +- .../q22.native_datafusion/extended.txt | 13 +- .../q22a.native_datafusion/extended.txt | 57 +- .../q27a.native_datafusion/extended.txt | 35 +- .../q34.native_datafusion/extended.txt | 13 +- .../q35.native_datafusion/extended.txt | 35 +- .../q35a.native_datafusion/extended.txt | 35 +- .../q36a.native_datafusion/extended.txt | 35 +- .../q47.native_datafusion/extended.txt | 29 +- .../q49.native_datafusion/extended.txt | 35 +- .../q51a.native_datafusion/extended.txt | 90 +- .../q57.native_datafusion/extended.txt | 29 +- .../q5a.native_datafusion/extended.txt | 200 ++- .../q6.native_datafusion/extended.txt | 45 +- .../q64.native_datafusion/extended.txt | 20 +- .../q67a.native_datafusion/extended.txt | 101 +- .../q70a.native_datafusion/extended.txt | 68 +- .../q72.native_datafusion/extended.txt | 13 +- .../q74.native_datafusion/extended.txt | 38 +- .../q75.native_datafusion/extended.txt | 56 +- .../q77a.native_datafusion/extended.txt | 200 ++- .../q78.native_datafusion/extended.txt | 29 +- .../q80a.native_datafusion/extended.txt | 101 +- .../q86a.native_datafusion/extended.txt | 35 +- .../q98.native_datafusion/extended.txt | 13 +- .../q10a.native_datafusion/extended.txt | 35 +- .../q11.native_datafusion/extended.txt | 38 +- .../q12.native_datafusion/extended.txt | 13 +- .../q14.native_datafusion/extended.txt | 221 ++- .../q14a.native_datafusion/extended.txt | 1322 ++++++++--------- .../q18a.native_datafusion/extended.txt | 57 +- .../q20.native_datafusion/extended.txt | 13 +- .../q22.native_datafusion/extended.txt | 13 +- .../q22a.native_datafusion/extended.txt | 57 +- .../q27a.native_datafusion/extended.txt | 35 +- .../q34.native_datafusion/extended.txt | 13 +- .../q35.native_datafusion/extended.txt | 35 +- .../q35a.native_datafusion/extended.txt | 35 +- .../q36a.native_datafusion/extended.txt | 35 +- .../q47.native_datafusion/extended.txt | 29 +- .../q49.native_datafusion/extended.txt | 35 +- .../q51a.native_datafusion/extended.txt | 90 +- .../q57.native_datafusion/extended.txt | 29 +- .../q5a.native_datafusion/extended.txt | 200 ++- .../q6.native_datafusion/extended.txt | 38 +- .../q64.native_datafusion/extended.txt | 20 +- .../q67a.native_datafusion/extended.txt | 101 +- .../q70a.native_datafusion/extended.txt | 68 +- .../q72.native_datafusion/extended.txt | 13 +- .../q74.native_datafusion/extended.txt | 38 +- .../q75.native_datafusion/extended.txt | 56 +- .../q77a.native_datafusion/extended.txt | 200 ++- .../q78.native_datafusion/extended.txt | 29 +- .../q80a.native_datafusion/extended.txt | 101 +- .../q86a.native_datafusion/extended.txt | 35 +- .../q98.native_datafusion/extended.txt | 13 +- 331 files changed, 7866 insertions(+), 9211 deletions(-) diff --git a/dev/diffs/3.4.3.diff b/dev/diffs/3.4.3.diff index 3d542c4a4c..d0d7d739ca 100644 --- a/dev/diffs/3.4.3.diff +++ b/dev/diffs/3.4.3.diff @@ -417,18 +417,26 @@ index daef11ae4d6..9f3cc9181f2 100644 assert(exchanges.size == 2) } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DynamicPartitionPruningSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DynamicPartitionPruningSuite.scala -index f33432ddb6f..8ec7285ed84 100644 +index f33432ddb6f..579a9c271de 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/DynamicPartitionPruningSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/DynamicPartitionPruningSuite.scala @@ -22,6 +22,7 @@ import org.scalatest.GivenWhenThen import org.apache.spark.sql.catalyst.expressions.{DynamicPruningExpression, Expression} import org.apache.spark.sql.catalyst.expressions.CodegenObjectFactoryMode._ import org.apache.spark.sql.catalyst.plans.ExistenceJoin -+import org.apache.spark.sql.comet.{CometNativeScanExec, CometScanExec} ++import org.apache.spark.sql.comet.{CometNativeScanExec, CometScanExec, CometSubqueryBroadcastExec} import org.apache.spark.sql.connector.catalog.{InMemoryTableCatalog, InMemoryTableWithV2FilterCatalog} import org.apache.spark.sql.execution._ import org.apache.spark.sql.execution.adaptive._ -@@ -262,6 +263,12 @@ abstract class DynamicPartitionPruningSuiteBase +@@ -193,6 +194,7 @@ abstract class DynamicPartitionPruningSuiteBase + } + val subqueryBroadcast = dpExprs.collect { + case InSubqueryExec(_, b: SubqueryBroadcastExec, _, _, _, _) => b ++ case InSubqueryExec(_, b: CometSubqueryBroadcastExec, _, _, _, _) => b + } + + val hasFilter = if (withSubquery) "Should" else "Shouldn't" +@@ -262,6 +264,12 @@ abstract class DynamicPartitionPruningSuiteBase case s: BatchScanExec => s.runtimeFilters.collect { case d: DynamicPruningExpression => d.child } @@ -441,7 +449,7 @@ index f33432ddb6f..8ec7285ed84 100644 case _ => Nil } } -@@ -1027,7 +1034,8 @@ abstract class DynamicPartitionPruningSuiteBase +@@ -1027,7 +1035,8 @@ abstract class DynamicPartitionPruningSuiteBase } } @@ -451,7 +459,7 @@ index f33432ddb6f..8ec7285ed84 100644 withSQLConf(SQLConf.DYNAMIC_PARTITION_PRUNING_REUSE_BROADCAST_ONLY.key -> "false", SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> "-1") { withTable("large", "dimTwo", "dimThree") { -@@ -1215,7 +1223,8 @@ abstract class DynamicPartitionPruningSuiteBase +@@ -1215,7 +1224,8 @@ abstract class DynamicPartitionPruningSuiteBase } test("SPARK-32509: Unused Dynamic Pruning filter shouldn't affect " + @@ -461,7 +469,7 @@ index f33432ddb6f..8ec7285ed84 100644 withSQLConf(SQLConf.DYNAMIC_PARTITION_PRUNING_REUSE_BROADCAST_ONLY.key -> "true") { withSQLConf(SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> "-1") { val df = sql( -@@ -1423,7 +1432,8 @@ abstract class DynamicPartitionPruningSuiteBase +@@ -1423,7 +1433,8 @@ abstract class DynamicPartitionPruningSuiteBase } } @@ -471,7 +479,7 @@ index f33432ddb6f..8ec7285ed84 100644 withSQLConf(SQLConf.DYNAMIC_PARTITION_PRUNING_REUSE_BROADCAST_ONLY.key -> "true") { val df = sql( """ WITH v as ( -@@ -1698,7 +1708,8 @@ abstract class DynamicPartitionPruningV1Suite extends DynamicPartitionPruningDat +@@ -1698,7 +1709,8 @@ abstract class DynamicPartitionPruningV1Suite extends DynamicPartitionPruningDat * Check the static scan metrics with and without DPP */ test("static scan metrics", @@ -481,7 +489,7 @@ index f33432ddb6f..8ec7285ed84 100644 withSQLConf(SQLConf.DYNAMIC_PARTITION_PRUNING_ENABLED.key -> "true", SQLConf.DYNAMIC_PARTITION_PRUNING_REUSE_BROADCAST_ONLY.key -> "false", SQLConf.EXCHANGE_REUSE_ENABLED.key -> "false") { -@@ -1729,6 +1740,8 @@ abstract class DynamicPartitionPruningV1Suite extends DynamicPartitionPruningDat +@@ -1729,6 +1741,8 @@ abstract class DynamicPartitionPruningV1Suite extends DynamicPartitionPruningDat case s: BatchScanExec => // we use f1 col for v2 tables due to schema pruning s.output.exists(_.exists(_.argString(maxFields = 100).contains("f1"))) @@ -1297,7 +1305,7 @@ index 4b3d3a4b805..56e1e0e6f16 100644 setupTestData() diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/RemoveRedundantProjectsSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/RemoveRedundantProjectsSuite.scala -index 9e9d717db3b..ec73082f458 100644 +index 9e9d717db3b..ad41c8f3901 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/RemoveRedundantProjectsSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/RemoveRedundantProjectsSuite.scala @@ -17,7 +17,10 @@ @@ -1324,17 +1332,7 @@ index 9e9d717db3b..ec73082f458 100644 assert(actual == expected) } } -@@ -112,7 +118,8 @@ abstract class RemoveRedundantProjectsSuiteBase - assertProjectExec(query, 1, 3) - } - -- test("join with ordering requirement") { -+ test("join with ordering requirement", -+ IgnoreComet("TODO: Support SubqueryBroadcastExec in Comet: #242")) { - val query = "select * from (select key, a, c, b from testView) as t1 join " + - "(select key, a, b, c from testView) as t2 on t1.key = t2.key where t2.a > 50" - assertProjectExec(query, 2, 2) -@@ -134,12 +141,21 @@ abstract class RemoveRedundantProjectsSuiteBase +@@ -134,12 +140,21 @@ abstract class RemoveRedundantProjectsSuiteBase val df = data.selectExpr("a", "b", "key", "explode(array(key, a, b)) as d").filter("d > 0") df.collect() val plan = df.queryExecution.executedPlan @@ -1361,7 +1359,7 @@ index 9e9d717db3b..ec73082f458 100644 case g @ GenerateExec(_, requiredChildOutput, _, _, child) => g.copy(requiredChildOutput = requiredChildOutput.reverse, child = ProjectExec(requiredChildOutput.reverse, child)) -@@ -151,6 +167,7 @@ abstract class RemoveRedundantProjectsSuiteBase +@@ -151,6 +166,7 @@ abstract class RemoveRedundantProjectsSuiteBase // The manually added ProjectExec node shouldn't be removed. assert(collectWithSubqueries(newExecutedPlan) { case p: ProjectExec => p diff --git a/dev/diffs/3.5.8.diff b/dev/diffs/3.5.8.diff index 6462f4f489..687a1b9e9a 100644 --- a/dev/diffs/3.5.8.diff +++ b/dev/diffs/3.5.8.diff @@ -398,18 +398,26 @@ index c4fb4fa943c..a04b23870a8 100644 assert(exchanges.size == 2) } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DynamicPartitionPruningSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DynamicPartitionPruningSuite.scala -index f33432ddb6f..8ec7285ed84 100644 +index f33432ddb6f..579a9c271de 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/DynamicPartitionPruningSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/DynamicPartitionPruningSuite.scala @@ -22,6 +22,7 @@ import org.scalatest.GivenWhenThen import org.apache.spark.sql.catalyst.expressions.{DynamicPruningExpression, Expression} import org.apache.spark.sql.catalyst.expressions.CodegenObjectFactoryMode._ import org.apache.spark.sql.catalyst.plans.ExistenceJoin -+import org.apache.spark.sql.comet.{CometNativeScanExec, CometScanExec} ++import org.apache.spark.sql.comet.{CometNativeScanExec, CometScanExec, CometSubqueryBroadcastExec} import org.apache.spark.sql.connector.catalog.{InMemoryTableCatalog, InMemoryTableWithV2FilterCatalog} import org.apache.spark.sql.execution._ import org.apache.spark.sql.execution.adaptive._ -@@ -262,6 +263,12 @@ abstract class DynamicPartitionPruningSuiteBase +@@ -193,6 +194,7 @@ abstract class DynamicPartitionPruningSuiteBase + } + val subqueryBroadcast = dpExprs.collect { + case InSubqueryExec(_, b: SubqueryBroadcastExec, _, _, _, _) => b ++ case InSubqueryExec(_, b: CometSubqueryBroadcastExec, _, _, _, _) => b + } + + val hasFilter = if (withSubquery) "Should" else "Shouldn't" +@@ -262,6 +264,12 @@ abstract class DynamicPartitionPruningSuiteBase case s: BatchScanExec => s.runtimeFilters.collect { case d: DynamicPruningExpression => d.child } @@ -422,7 +430,7 @@ index f33432ddb6f..8ec7285ed84 100644 case _ => Nil } } -@@ -1027,7 +1034,8 @@ abstract class DynamicPartitionPruningSuiteBase +@@ -1027,7 +1035,8 @@ abstract class DynamicPartitionPruningSuiteBase } } @@ -432,7 +440,7 @@ index f33432ddb6f..8ec7285ed84 100644 withSQLConf(SQLConf.DYNAMIC_PARTITION_PRUNING_REUSE_BROADCAST_ONLY.key -> "false", SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> "-1") { withTable("large", "dimTwo", "dimThree") { -@@ -1215,7 +1223,8 @@ abstract class DynamicPartitionPruningSuiteBase +@@ -1215,7 +1224,8 @@ abstract class DynamicPartitionPruningSuiteBase } test("SPARK-32509: Unused Dynamic Pruning filter shouldn't affect " + @@ -442,7 +450,7 @@ index f33432ddb6f..8ec7285ed84 100644 withSQLConf(SQLConf.DYNAMIC_PARTITION_PRUNING_REUSE_BROADCAST_ONLY.key -> "true") { withSQLConf(SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> "-1") { val df = sql( -@@ -1423,7 +1432,8 @@ abstract class DynamicPartitionPruningSuiteBase +@@ -1423,7 +1433,8 @@ abstract class DynamicPartitionPruningSuiteBase } } @@ -452,7 +460,7 @@ index f33432ddb6f..8ec7285ed84 100644 withSQLConf(SQLConf.DYNAMIC_PARTITION_PRUNING_REUSE_BROADCAST_ONLY.key -> "true") { val df = sql( """ WITH v as ( -@@ -1698,7 +1708,8 @@ abstract class DynamicPartitionPruningV1Suite extends DynamicPartitionPruningDat +@@ -1698,7 +1709,8 @@ abstract class DynamicPartitionPruningV1Suite extends DynamicPartitionPruningDat * Check the static scan metrics with and without DPP */ test("static scan metrics", @@ -462,7 +470,7 @@ index f33432ddb6f..8ec7285ed84 100644 withSQLConf(SQLConf.DYNAMIC_PARTITION_PRUNING_ENABLED.key -> "true", SQLConf.DYNAMIC_PARTITION_PRUNING_REUSE_BROADCAST_ONLY.key -> "false", SQLConf.EXCHANGE_REUSE_ENABLED.key -> "false") { -@@ -1729,6 +1740,8 @@ abstract class DynamicPartitionPruningV1Suite extends DynamicPartitionPruningDat +@@ -1729,6 +1741,8 @@ abstract class DynamicPartitionPruningV1Suite extends DynamicPartitionPruningDat case s: BatchScanExec => // we use f1 col for v2 tables due to schema pruning s.output.exists(_.exists(_.argString(maxFields = 100).contains("f1"))) @@ -1248,17 +1256,16 @@ index de24b8c82b0..1f835481290 100644 setupTestData() diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/RemoveRedundantProjectsSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/RemoveRedundantProjectsSuite.scala -index 9e9d717db3b..cdd1042a880 100644 +index 9e9d717db3b..73de2b84938 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/RemoveRedundantProjectsSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/RemoveRedundantProjectsSuite.scala @@ -17,7 +17,10 @@ package org.apache.spark.sql.execution --import org.apache.spark.sql.{DataFrame, QueryTest, Row} +import org.apache.comet.CometConf + -+import org.apache.spark.sql.{DataFrame, IgnoreCometNativeDataFusion, QueryTest, Row} + import org.apache.spark.sql.{DataFrame, QueryTest, Row} +import org.apache.spark.sql.comet.CometProjectExec import org.apache.spark.sql.connector.SimpleWritableDataSource import org.apache.spark.sql.execution.adaptive.{AdaptiveSparkPlanHelper, DisableAdaptiveExecutionSuite, EnableAdaptiveExecutionSuite} @@ -1275,17 +1282,7 @@ index 9e9d717db3b..cdd1042a880 100644 assert(actual == expected) } } -@@ -112,7 +118,8 @@ abstract class RemoveRedundantProjectsSuiteBase - assertProjectExec(query, 1, 3) - } - -- test("join with ordering requirement") { -+ test("join with ordering requirement", -+ IgnoreCometNativeDataFusion("https://github.com/apache/datafusion-comet/issues/4014")) { - val query = "select * from (select key, a, c, b from testView) as t1 join " + - "(select key, a, b, c from testView) as t2 on t1.key = t2.key where t2.a > 50" - assertProjectExec(query, 2, 2) -@@ -134,12 +141,26 @@ abstract class RemoveRedundantProjectsSuiteBase +@@ -134,12 +140,26 @@ abstract class RemoveRedundantProjectsSuiteBase val df = data.selectExpr("a", "b", "key", "explode(array(key, a, b)) as d").filter("d > 0") df.collect() val plan = df.queryExecution.executedPlan @@ -1314,7 +1311,7 @@ index 9e9d717db3b..cdd1042a880 100644 case g @ GenerateExec(_, requiredChildOutput, _, _, child) => g.copy(requiredChildOutput = requiredChildOutput.reverse, child = ProjectExec(requiredChildOutput.reverse, child)) -@@ -151,6 +172,7 @@ abstract class RemoveRedundantProjectsSuiteBase +@@ -151,6 +171,7 @@ abstract class RemoveRedundantProjectsSuiteBase // The manually added ProjectExec node shouldn't be removed. assert(collectWithSubqueries(newExecutedPlan) { case p: ProjectExec => p diff --git a/dev/diffs/4.0.1.diff b/dev/diffs/4.0.1.diff index e88165da6d..968988cc7d 100644 --- a/dev/diffs/4.0.1.diff +++ b/dev/diffs/4.0.1.diff @@ -589,18 +589,26 @@ index 81713c777bc..b5f92ed9742 100644 assert(exchanges.size == 2) } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DynamicPartitionPruningSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DynamicPartitionPruningSuite.scala -index 2c24cc7d570..697ea7f8a60 100644 +index 2c24cc7d570..fd3c85eef87 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/DynamicPartitionPruningSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/DynamicPartitionPruningSuite.scala @@ -22,6 +22,7 @@ import org.scalatest.GivenWhenThen import org.apache.spark.sql.catalyst.expressions.{DynamicPruningExpression, Expression} import org.apache.spark.sql.catalyst.expressions.CodegenObjectFactoryMode._ import org.apache.spark.sql.catalyst.plans.ExistenceJoin -+import org.apache.spark.sql.comet.{CometNativeScanExec, CometScanExec} ++import org.apache.spark.sql.comet.{CometNativeScanExec, CometScanExec, CometSubqueryBroadcastExec} import org.apache.spark.sql.connector.catalog.{InMemoryTableCatalog, InMemoryTableWithV2FilterCatalog} import org.apache.spark.sql.execution._ import org.apache.spark.sql.execution.adaptive._ -@@ -262,6 +263,12 @@ abstract class DynamicPartitionPruningSuiteBase +@@ -193,6 +194,7 @@ abstract class DynamicPartitionPruningSuiteBase + } + val subqueryBroadcast = dpExprs.collect { + case InSubqueryExec(_, b: SubqueryBroadcastExec, _, _, _, _) => b ++ case InSubqueryExec(_, b: CometSubqueryBroadcastExec, _, _, _, _) => b + } + + val hasFilter = if (withSubquery) "Should" else "Shouldn't" +@@ -262,6 +264,12 @@ abstract class DynamicPartitionPruningSuiteBase case s: BatchScanExec => s.runtimeFilters.collect { case d: DynamicPruningExpression => d.child } @@ -613,7 +621,7 @@ index 2c24cc7d570..697ea7f8a60 100644 case _ => Nil } } -@@ -1027,7 +1034,8 @@ abstract class DynamicPartitionPruningSuiteBase +@@ -1027,7 +1035,8 @@ abstract class DynamicPartitionPruningSuiteBase } } @@ -623,7 +631,7 @@ index 2c24cc7d570..697ea7f8a60 100644 withSQLConf(SQLConf.DYNAMIC_PARTITION_PRUNING_REUSE_BROADCAST_ONLY.key -> "false", SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> "-1") { withTable("large", "dimTwo", "dimThree") { -@@ -1151,7 +1159,8 @@ abstract class DynamicPartitionPruningSuiteBase +@@ -1151,7 +1160,8 @@ abstract class DynamicPartitionPruningSuiteBase } } @@ -633,7 +641,7 @@ index 2c24cc7d570..697ea7f8a60 100644 withSQLConf(SQLConf.DYNAMIC_PARTITION_PRUNING_REUSE_BROADCAST_ONLY.key -> "true", SQLConf.ADAPTIVE_OPTIMIZER_EXCLUDED_RULES.key -> AQEPropagateEmptyRelation.ruleName, SQLConf.ANSI_ENABLED.key -> "false" // ANSI mode doesn't support "String + String" -@@ -1215,7 +1224,8 @@ abstract class DynamicPartitionPruningSuiteBase +@@ -1215,7 +1225,8 @@ abstract class DynamicPartitionPruningSuiteBase } test("SPARK-32509: Unused Dynamic Pruning filter shouldn't affect " + @@ -643,7 +651,7 @@ index 2c24cc7d570..697ea7f8a60 100644 withSQLConf(SQLConf.DYNAMIC_PARTITION_PRUNING_REUSE_BROADCAST_ONLY.key -> "true") { withSQLConf(SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> "-1") { val df = sql( -@@ -1330,6 +1340,7 @@ abstract class DynamicPartitionPruningSuiteBase +@@ -1330,6 +1341,7 @@ abstract class DynamicPartitionPruningSuiteBase } test("Subquery reuse across the whole plan", @@ -651,7 +659,7 @@ index 2c24cc7d570..697ea7f8a60 100644 DisableAdaptiveExecution("DPP in AQE must reuse broadcast")) { withSQLConf(SQLConf.DYNAMIC_PARTITION_PRUNING_ENABLED.key -> "true", SQLConf.DYNAMIC_PARTITION_PRUNING_REUSE_BROADCAST_ONLY.key -> "false", -@@ -1424,7 +1435,8 @@ abstract class DynamicPartitionPruningSuiteBase +@@ -1424,7 +1436,8 @@ abstract class DynamicPartitionPruningSuiteBase } } @@ -661,7 +669,7 @@ index 2c24cc7d570..697ea7f8a60 100644 withSQLConf(SQLConf.DYNAMIC_PARTITION_PRUNING_REUSE_BROADCAST_ONLY.key -> "true") { val df = sql( """ WITH v as ( -@@ -1699,7 +1711,8 @@ abstract class DynamicPartitionPruningV1Suite extends DynamicPartitionPruningDat +@@ -1699,7 +1712,8 @@ abstract class DynamicPartitionPruningV1Suite extends DynamicPartitionPruningDat * Check the static scan metrics with and without DPP */ test("static scan metrics", @@ -671,7 +679,7 @@ index 2c24cc7d570..697ea7f8a60 100644 withSQLConf(SQLConf.DYNAMIC_PARTITION_PRUNING_ENABLED.key -> "true", SQLConf.DYNAMIC_PARTITION_PRUNING_REUSE_BROADCAST_ONLY.key -> "false", SQLConf.EXCHANGE_REUSE_ENABLED.key -> "false") { -@@ -1730,6 +1743,8 @@ abstract class DynamicPartitionPruningV1Suite extends DynamicPartitionPruningDat +@@ -1730,6 +1744,8 @@ abstract class DynamicPartitionPruningV1Suite extends DynamicPartitionPruningDat case s: BatchScanExec => // we use f1 col for v2 tables due to schema pruning s.output.exists(_.exists(_.argString(maxFields = 100).contains("f1"))) @@ -1755,7 +1763,7 @@ index 47d5ff67b84..8dc8f65d4b1 100644 withSQLConf(SQLConf.ADAPTIVE_EXECUTION_ENABLED.key -> "true", SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> "-1", diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/RemoveRedundantProjectsSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/RemoveRedundantProjectsSuite.scala -index b5bac8079c4..9420dbdb936 100644 +index b5bac8079c4..1606dd057e0 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/RemoveRedundantProjectsSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/RemoveRedundantProjectsSuite.scala @@ -17,7 +17,10 @@ @@ -1782,17 +1790,7 @@ index b5bac8079c4..9420dbdb936 100644 assert(actual == expected) } } -@@ -112,7 +118,8 @@ abstract class RemoveRedundantProjectsSuiteBase - assertProjectExec(query, 1, 3) - } - -- test("join with ordering requirement") { -+ test("join with ordering requirement", -+ IgnoreComet("TODO: Support SubqueryBroadcastExec in Comet: #242")) { - val query = "select * from (select key, a, c, b from testView) as t1 join " + - "(select key, a, b, c from testView) as t2 on t1.key = t2.key where t2.a > 50" - assertProjectExec(query, 2, 2) -@@ -134,12 +141,25 @@ abstract class RemoveRedundantProjectsSuiteBase +@@ -134,12 +140,25 @@ abstract class RemoveRedundantProjectsSuiteBase val df = data.selectExpr("a", "b", "key", "explode(array(key, a, b)) as d").filter("d > 0") df.collect() val plan = df.queryExecution.executedPlan @@ -1820,7 +1818,7 @@ index b5bac8079c4..9420dbdb936 100644 case g @ GenerateExec(_, requiredChildOutput, _, _, child) => g.copy(requiredChildOutput = requiredChildOutput.reverse, child = ProjectExec(requiredChildOutput.reverse, child)) -@@ -151,6 +171,7 @@ abstract class RemoveRedundantProjectsSuiteBase +@@ -151,6 +170,7 @@ abstract class RemoveRedundantProjectsSuiteBase // The manually added ProjectExec node shouldn't be removed. assert(collectWithSubqueries(newExecutedPlan) { case p: ProjectExec => p diff --git a/spark/src/main/scala/org/apache/comet/rules/CometScanRule.scala b/spark/src/main/scala/org/apache/comet/rules/CometScanRule.scala index befff09bc0..878669d625 100644 --- a/spark/src/main/scala/org/apache/comet/rules/CometScanRule.scala +++ b/spark/src/main/scala/org/apache/comet/rules/CometScanRule.scala @@ -646,7 +646,15 @@ case class CometScanRule(session: SparkSession) private def isDynamicPruningFilter(e: Expression): Boolean = e.exists(_.isInstanceOf[PlanExpression[_]]) - /** Detects AQE DPP (SubqueryAdaptiveBroadcastExec), as opposed to non-AQE DPP. */ + /** + * Detects AQE DPP (SubqueryAdaptiveBroadcastExec), as opposed to non-AQE DPP. + * + * Non-AQE DPP (PlanDynamicPruningFilters) runs before Comet rules and produces + * SubqueryBroadcastExec/SubqueryExec which Spark's execution framework resolves. AQE DPP + * (PlanAdaptiveDynamicPruningFilters) runs after Comet rules and searches for + * BroadcastHashJoinExec. It doesn't recognize Comet operators, so it can't create DPP filters + * correctly. + */ private def isAqeDynamicPruningFilter(e: Expression): Boolean = e.exists { case sub: InSubqueryExec => sub.plan.isInstanceOf[SubqueryAdaptiveBroadcastExec] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q1.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q1.native_datafusion/extended.txt index 935c103614..5875fa3a5e 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q1.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q1.native_datafusion/extended.txt @@ -14,12 +14,11 @@ CometNativeColumnarToRow : : : +- CometBroadcastHashJoin : : : :- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.store_returns - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometSubqueryBroadcast + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter @@ -36,12 +35,11 @@ CometNativeColumnarToRow : : +- CometBroadcastHashJoin : : :- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.store_returns - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometSubqueryBroadcast + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- CometBroadcastExchange : : +- CometProject : : +- CometFilter @@ -55,4 +53,4 @@ CometNativeColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.customer -Comet accelerated 49 out of 53 eligible operators (92%). Final plan contains 3 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 51 out of 53 eligible operators (96%). Final plan contains 1 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q10.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q10.native_datafusion/extended.txt index 2785c3d0b1..2e79f6f464 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q10.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q10.native_datafusion/extended.txt @@ -19,12 +19,11 @@ TakeOrderedAndProject : : : : +- CometProject : : : : +- CometBroadcastHashJoin : : : : :- CometNativeScan parquet spark_catalog.default.store_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- CometSubqueryBroadcast + : : : : : +- CometBroadcastExchange + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : +- CometBroadcastExchange : : : : +- CometProject : : : : +- CometFilter @@ -34,12 +33,11 @@ TakeOrderedAndProject : : : +- CometProject : : : +- CometBroadcastHashJoin : : : :- CometNativeScan parquet spark_catalog.default.web_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometSubqueryBroadcast + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter @@ -49,12 +47,11 @@ TakeOrderedAndProject : : +- CometProject : : +- CometBroadcastHashJoin : : :- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometSubqueryBroadcast + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- CometBroadcastExchange : : +- CometProject : : +- CometFilter @@ -70,4 +67,4 @@ TakeOrderedAndProject +- CometFilter +- CometNativeScan parquet spark_catalog.default.customer_demographics -Comet accelerated 41 out of 62 eligible operators (66%). Final plan contains 9 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 44 out of 62 eligible operators (70%). Final plan contains 6 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q11.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q11.native_datafusion/extended.txt index 6b081dd35b..edda694b73 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q11.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q11.native_datafusion/extended.txt @@ -20,11 +20,10 @@ CometNativeColumnarToRow : : : : +- CometBroadcastExchange : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometSubqueryBroadcast + : : : : +- CometBroadcastExchange + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- CometBroadcastExchange : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -42,11 +41,10 @@ CometNativeColumnarToRow : : : +- CometBroadcastExchange : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometSubqueryBroadcast + : : : +- CometBroadcastExchange + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- CometBroadcastExchange : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -65,11 +63,10 @@ CometNativeColumnarToRow : : +- CometBroadcastExchange : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometSubqueryBroadcast + : : +- CometBroadcastExchange + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim : +- CometBroadcastExchange : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -87,13 +84,12 @@ CometNativeColumnarToRow : +- CometBroadcastExchange : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.web_sales - : +- SubqueryBroadcast - : +- BroadcastExchange - : +- CometNativeColumnarToRow - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometSubqueryBroadcast + : +- CometBroadcastExchange + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.date_dim +- CometBroadcastExchange +- CometFilter +- CometNativeScan parquet spark_catalog.default.date_dim -Comet accelerated 84 out of 92 eligible operators (91%). Final plan contains 5 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 88 out of 92 eligible operators (95%). Final plan contains 1 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q12.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q12.native_datafusion/extended.txt index a8c2531a38..a6cc526735 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q12.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q12.native_datafusion/extended.txt @@ -13,12 +13,11 @@ TakeOrderedAndProject : +- CometBroadcastHashJoin : :- CometFilter : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometSubqueryBroadcast + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim : +- CometBroadcastExchange : +- CometProject : +- CometFilter @@ -28,4 +27,4 @@ TakeOrderedAndProject +- CometFilter +- CometNativeScan parquet spark_catalog.default.date_dim -Comet accelerated 22 out of 27 eligible operators (81%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 23 out of 27 eligible operators (85%). Final plan contains 1 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q13.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q13.native_datafusion/extended.txt index 42ea1c4966..33c2dd1bbc 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q13.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q13.native_datafusion/extended.txt @@ -14,12 +14,11 @@ CometNativeColumnarToRow : : : : +- CometBroadcastHashJoin : : : : :- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- CometSubqueryBroadcast + : : : : : +- CometBroadcastExchange + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : +- CometBroadcastExchange : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.store @@ -39,4 +38,4 @@ CometNativeColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.household_demographics -Comet accelerated 36 out of 38 eligible operators (94%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 37 out of 38 eligible operators (97%). Final plan contains 1 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q14a.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q14a.native_datafusion/extended.txt index 0dcd47f57f..d2a44e9c52 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q14a.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q14a.native_datafusion/extended.txt @@ -16,12 +16,11 @@ CometNativeColumnarToRow : : :- CometProject : : : +- CometBroadcastHashJoin : : : :- CometNativeScan parquet spark_catalog.default.store_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometSubqueryBroadcast + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter @@ -29,12 +28,11 @@ CometNativeColumnarToRow : : :- CometProject : : : +- CometBroadcastHashJoin : : : :- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometSubqueryBroadcast + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter @@ -42,12 +40,11 @@ CometNativeColumnarToRow : : +- CometProject : : +- CometBroadcastHashJoin : : :- CometNativeScan parquet spark_catalog.default.web_sales - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometSubqueryBroadcast + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- CometBroadcastExchange : : +- CometProject : : +- CometFilter @@ -62,12 +59,11 @@ CometNativeColumnarToRow : : :- CometBroadcastHashJoin : : : :- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometSubqueryBroadcast + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometBroadcastHashJoin @@ -84,12 +80,11 @@ CometNativeColumnarToRow : : : : : +- CometBroadcastHashJoin : : : : : :- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometNativeColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : +- CometSubqueryBroadcast + : : : : : : +- CometBroadcastExchange + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : : +- CometBroadcastExchange : : : : : +- CometBroadcastHashJoin : : : : : :- CometFilter @@ -101,12 +96,11 @@ CometNativeColumnarToRow : : : : : : +- CometBroadcastHashJoin : : : : : : :- CometFilter : : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : : : : +- SubqueryBroadcast - : : : : : : : +- BroadcastExchange - : : : : : : : +- CometNativeColumnarToRow - : : : : : : : +- CometProject - : : : : : : : +- CometFilter - : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : : +- CometSubqueryBroadcast + : : : : : : : +- CometBroadcastExchange + : : : : : : : +- CometProject + : : : : : : : +- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : : : +- CometBroadcastExchange : : : : : : +- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.item @@ -125,12 +119,11 @@ CometNativeColumnarToRow : : : : +- CometBroadcastHashJoin : : : : :- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- CometSubqueryBroadcast + : : : : : +- CometBroadcastExchange + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : +- CometBroadcastExchange : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.item @@ -158,12 +151,11 @@ CometNativeColumnarToRow : : : : +- CometBroadcastHashJoin : : : : :- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- CometSubqueryBroadcast + : : : : : +- CometBroadcastExchange + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : +- CometBroadcastExchange : : : : +- CometBroadcastHashJoin : : : : :- CometFilter @@ -175,12 +167,11 @@ CometNativeColumnarToRow : : : : : +- CometBroadcastHashJoin : : : : : :- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometNativeColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : +- CometSubqueryBroadcast + : : : : : : +- CometBroadcastExchange + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : : +- CometBroadcastExchange : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.item @@ -199,12 +190,11 @@ CometNativeColumnarToRow : : : +- CometBroadcastHashJoin : : : :- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometSubqueryBroadcast + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- CometBroadcastExchange : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.item @@ -229,12 +219,11 @@ CometNativeColumnarToRow : : :- CometBroadcastHashJoin : : : :- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometSubqueryBroadcast + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometBroadcastHashJoin @@ -251,12 +240,11 @@ CometNativeColumnarToRow : : : : : +- CometBroadcastHashJoin : : : : : :- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometNativeColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : +- CometSubqueryBroadcast + : : : : : : +- CometBroadcastExchange + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : : +- CometBroadcastExchange : : : : : +- CometBroadcastHashJoin : : : : : :- CometFilter @@ -268,12 +256,11 @@ CometNativeColumnarToRow : : : : : : +- CometBroadcastHashJoin : : : : : : :- CometFilter : : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : : : : +- SubqueryBroadcast - : : : : : : : +- BroadcastExchange - : : : : : : : +- CometNativeColumnarToRow - : : : : : : : +- CometProject - : : : : : : : +- CometFilter - : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : : +- CometSubqueryBroadcast + : : : : : : : +- CometBroadcastExchange + : : : : : : : +- CometProject + : : : : : : : +- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : : : +- CometBroadcastExchange : : : : : : +- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.item @@ -292,12 +279,11 @@ CometNativeColumnarToRow : : : : +- CometBroadcastHashJoin : : : : :- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- CometSubqueryBroadcast + : : : : : +- CometBroadcastExchange + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : +- CometBroadcastExchange : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.item @@ -325,12 +311,11 @@ CometNativeColumnarToRow : : : : +- CometBroadcastHashJoin : : : : :- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- CometSubqueryBroadcast + : : : : : +- CometBroadcastExchange + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : +- CometBroadcastExchange : : : : +- CometBroadcastHashJoin : : : : :- CometFilter @@ -342,12 +327,11 @@ CometNativeColumnarToRow : : : : : +- CometBroadcastHashJoin : : : : : :- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometNativeColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : +- CometSubqueryBroadcast + : : : : : : +- CometBroadcastExchange + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : : +- CometBroadcastExchange : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.item @@ -366,12 +350,11 @@ CometNativeColumnarToRow : : : +- CometBroadcastHashJoin : : : :- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometSubqueryBroadcast + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- CometBroadcastExchange : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.item @@ -396,12 +379,11 @@ CometNativeColumnarToRow : :- CometBroadcastHashJoin : : :- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometSubqueryBroadcast + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- CometBroadcastExchange : : +- CometProject : : +- CometBroadcastHashJoin @@ -418,12 +400,11 @@ CometNativeColumnarToRow : : : : +- CometBroadcastHashJoin : : : : :- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- CometSubqueryBroadcast + : : : : : +- CometBroadcastExchange + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : +- CometBroadcastExchange : : : : +- CometBroadcastHashJoin : : : : :- CometFilter @@ -435,12 +416,11 @@ CometNativeColumnarToRow : : : : : +- CometBroadcastHashJoin : : : : : :- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometNativeColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : +- CometSubqueryBroadcast + : : : : : : +- CometBroadcastExchange + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : : +- CometBroadcastExchange : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.item @@ -459,12 +439,11 @@ CometNativeColumnarToRow : : : +- CometBroadcastHashJoin : : : :- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometSubqueryBroadcast + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- CometBroadcastExchange : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.item @@ -492,12 +471,11 @@ CometNativeColumnarToRow : : : +- CometBroadcastHashJoin : : : :- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometSubqueryBroadcast + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- CometBroadcastExchange : : : +- CometBroadcastHashJoin : : : :- CometFilter @@ -509,12 +487,11 @@ CometNativeColumnarToRow : : : : +- CometBroadcastHashJoin : : : : :- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- CometSubqueryBroadcast + : : : : : +- CometBroadcastExchange + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : +- CometBroadcastExchange : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.item @@ -533,12 +510,11 @@ CometNativeColumnarToRow : : +- CometBroadcastHashJoin : : :- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometSubqueryBroadcast + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- CometBroadcastExchange : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.item @@ -551,4 +527,4 @@ CometNativeColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.date_dim -Comet accelerated 475 out of 526 eligible operators (90%). Final plan contains 26 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 499 out of 526 eligible operators (94%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q14b.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q14b.native_datafusion/extended.txt index 50139381f2..2d20c40e4b 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q14b.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q14b.native_datafusion/extended.txt @@ -11,12 +11,11 @@ CometNativeColumnarToRow : : :- CometProject : : : +- CometBroadcastHashJoin : : : :- CometNativeScan parquet spark_catalog.default.store_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometSubqueryBroadcast + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter @@ -24,12 +23,11 @@ CometNativeColumnarToRow : : :- CometProject : : : +- CometBroadcastHashJoin : : : :- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometSubqueryBroadcast + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter @@ -37,12 +35,11 @@ CometNativeColumnarToRow : : +- CometProject : : +- CometBroadcastHashJoin : : :- CometNativeScan parquet spark_catalog.default.web_sales - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometSubqueryBroadcast + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- CometBroadcastExchange : : +- CometProject : : +- CometFilter @@ -57,17 +54,16 @@ CometNativeColumnarToRow : : :- CometBroadcastHashJoin : : : :- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : : +- Subquery - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometSubqueryBroadcast + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : : +- Subquery + : : : : : +- CometNativeColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometBroadcastHashJoin @@ -84,12 +80,11 @@ CometNativeColumnarToRow : : : : : +- CometBroadcastHashJoin : : : : : :- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometNativeColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : +- CometSubqueryBroadcast + : : : : : : +- CometBroadcastExchange + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : : +- CometBroadcastExchange : : : : : +- CometBroadcastHashJoin : : : : : :- CometFilter @@ -101,12 +96,11 @@ CometNativeColumnarToRow : : : : : : +- CometBroadcastHashJoin : : : : : : :- CometFilter : : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : : : : +- SubqueryBroadcast - : : : : : : : +- BroadcastExchange - : : : : : : : +- CometNativeColumnarToRow - : : : : : : : +- CometProject - : : : : : : : +- CometFilter - : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : : +- CometSubqueryBroadcast + : : : : : : : +- CometBroadcastExchange + : : : : : : : +- CometProject + : : : : : : : +- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : : : +- CometBroadcastExchange : : : : : : +- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.item @@ -125,12 +119,11 @@ CometNativeColumnarToRow : : : : +- CometBroadcastHashJoin : : : : :- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- CometSubqueryBroadcast + : : : : : +- CometBroadcastExchange + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : +- CometBroadcastExchange : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.item @@ -158,12 +151,11 @@ CometNativeColumnarToRow : : : : +- CometBroadcastHashJoin : : : : :- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- CometSubqueryBroadcast + : : : : : +- CometBroadcastExchange + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : +- CometBroadcastExchange : : : : +- CometBroadcastHashJoin : : : : :- CometFilter @@ -175,12 +167,11 @@ CometNativeColumnarToRow : : : : : +- CometBroadcastHashJoin : : : : : :- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometNativeColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : +- CometSubqueryBroadcast + : : : : : : +- CometBroadcastExchange + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : : +- CometBroadcastExchange : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.item @@ -199,12 +190,11 @@ CometNativeColumnarToRow : : : +- CometBroadcastHashJoin : : : :- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometSubqueryBroadcast + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- CometBroadcastExchange : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.item @@ -215,7 +205,11 @@ CometNativeColumnarToRow : +- CometBroadcastExchange : +- CometProject : +- CometFilter - : : +- ReusedSubquery + : : +- Subquery + : : +- CometNativeColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim : +- CometNativeScan parquet spark_catalog.default.date_dim +- CometBroadcastExchange +- CometFilter @@ -230,17 +224,16 @@ CometNativeColumnarToRow : :- CometBroadcastHashJoin : : :- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : : +- Subquery - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometSubqueryBroadcast + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : : +- Subquery + : : : : +- CometNativeColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- CometBroadcastExchange : : +- CometProject : : +- CometBroadcastHashJoin @@ -257,12 +250,11 @@ CometNativeColumnarToRow : : : : +- CometBroadcastHashJoin : : : : :- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- CometSubqueryBroadcast + : : : : : +- CometBroadcastExchange + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : +- CometBroadcastExchange : : : : +- CometBroadcastHashJoin : : : : :- CometFilter @@ -274,12 +266,11 @@ CometNativeColumnarToRow : : : : : +- CometBroadcastHashJoin : : : : : :- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometNativeColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : +- CometSubqueryBroadcast + : : : : : : +- CometBroadcastExchange + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : : +- CometBroadcastExchange : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.item @@ -298,12 +289,11 @@ CometNativeColumnarToRow : : : +- CometBroadcastHashJoin : : : :- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometSubqueryBroadcast + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- CometBroadcastExchange : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.item @@ -331,12 +321,11 @@ CometNativeColumnarToRow : : : +- CometBroadcastHashJoin : : : :- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometSubqueryBroadcast + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- CometBroadcastExchange : : : +- CometBroadcastHashJoin : : : :- CometFilter @@ -348,12 +337,11 @@ CometNativeColumnarToRow : : : : +- CometBroadcastHashJoin : : : : :- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- CometSubqueryBroadcast + : : : : : +- CometBroadcastExchange + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : +- CometBroadcastExchange : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.item @@ -372,12 +360,11 @@ CometNativeColumnarToRow : : +- CometBroadcastHashJoin : : :- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometSubqueryBroadcast + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- CometBroadcastExchange : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.item @@ -388,7 +375,11 @@ CometNativeColumnarToRow +- CometBroadcastExchange +- CometProject +- CometFilter - : +- ReusedSubquery + : +- Subquery + : +- CometNativeColumnarToRow + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.date_dim +- CometNativeScan parquet spark_catalog.default.date_dim -Comet accelerated 331 out of 371 eligible operators (89%). Final plan contains 21 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 354 out of 377 eligible operators (93%). Final plan contains 6 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q15.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q15.native_datafusion/extended.txt index 42432bdc12..da1e91c969 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q15.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q15.native_datafusion/extended.txt @@ -11,12 +11,11 @@ CometNativeColumnarToRow : : +- CometBroadcastHashJoin : : :- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometSubqueryBroadcast + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- CometBroadcastExchange : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.customer @@ -29,4 +28,4 @@ CometNativeColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.date_dim -Comet accelerated 26 out of 28 eligible operators (92%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 27 out of 28 eligible operators (96%). Final plan contains 1 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q17.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q17.native_datafusion/extended.txt index f58c012650..2650cd8f1a 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q17.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q17.native_datafusion/extended.txt @@ -19,30 +19,27 @@ CometNativeColumnarToRow : : : : : : +- CometBroadcastHashJoin : : : : : : :- CometFilter : : : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : : : : : +- SubqueryBroadcast - : : : : : : : +- BroadcastExchange - : : : : : : : +- CometNativeColumnarToRow - : : : : : : : +- CometProject - : : : : : : : +- CometFilter - : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : : +- CometSubqueryBroadcast + : : : : : : : +- CometBroadcastExchange + : : : : : : : +- CometProject + : : : : : : : +- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : : : +- CometBroadcastExchange : : : : : : +- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.store_returns - : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometNativeColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : +- CometSubqueryBroadcast + : : : : : : +- CometBroadcastExchange + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : : +- CometBroadcastExchange : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- CometSubqueryBroadcast + : : : : : +- CometBroadcastExchange + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : +- CometBroadcastExchange : : : : +- CometProject : : : : +- CometFilter @@ -64,4 +61,4 @@ CometNativeColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.item -Comet accelerated 55 out of 61 eligible operators (90%). Final plan contains 4 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 58 out of 61 eligible operators (95%). Final plan contains 1 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q18.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q18.native_datafusion/extended.txt index b3485a3421..4040a49d5a 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q18.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q18.native_datafusion/extended.txt @@ -18,12 +18,11 @@ CometNativeColumnarToRow : : : : : +- CometBroadcastHashJoin : : : : : :- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometNativeColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : +- CometSubqueryBroadcast + : : : : : : +- CometBroadcastExchange + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : : +- CometBroadcastExchange : : : : : +- CometProject : : : : : +- CometFilter @@ -48,4 +47,4 @@ CometNativeColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.item -Comet accelerated 45 out of 47 eligible operators (95%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 46 out of 47 eligible operators (97%). Final plan contains 1 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q20.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q20.native_datafusion/extended.txt index c4f54f4289..24498162ca 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q20.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q20.native_datafusion/extended.txt @@ -13,12 +13,11 @@ TakeOrderedAndProject : +- CometBroadcastHashJoin : :- CometFilter : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometSubqueryBroadcast + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim : +- CometBroadcastExchange : +- CometProject : +- CometFilter @@ -28,4 +27,4 @@ TakeOrderedAndProject +- CometFilter +- CometNativeScan parquet spark_catalog.default.date_dim -Comet accelerated 22 out of 27 eligible operators (81%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 23 out of 27 eligible operators (85%). Final plan contains 1 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q21.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q21.native_datafusion/extended.txt index 8522eb4ab9..9cbd136255 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q21.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q21.native_datafusion/extended.txt @@ -12,11 +12,10 @@ CometNativeColumnarToRow : : +- CometBroadcastHashJoin : : :- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.inventory - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometSubqueryBroadcast + : : : +- CometBroadcastExchange + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- CometBroadcastExchange : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.warehouse @@ -28,4 +27,4 @@ CometNativeColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.date_dim -Comet accelerated 25 out of 27 eligible operators (92%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 26 out of 27 eligible operators (96%). Final plan contains 1 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q22.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q22.native_datafusion/extended.txt index 6b78d21629..36a0eb0c8a 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q22.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q22.native_datafusion/extended.txt @@ -12,12 +12,11 @@ CometNativeColumnarToRow : : +- CometBroadcastHashJoin : : :- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.inventory - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometSubqueryBroadcast + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- CometBroadcastExchange : : +- CometProject : : +- CometFilter @@ -30,4 +29,4 @@ CometNativeColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.warehouse -Comet accelerated 27 out of 29 eligible operators (93%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 28 out of 29 eligible operators (96%). Final plan contains 1 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q23a.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q23a.native_datafusion/extended.txt index 8a9990d536..3e46e4a329 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q23a.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q23a.native_datafusion/extended.txt @@ -12,12 +12,11 @@ CometNativeColumnarToRow : : : +- CometProject : : : +- CometBroadcastHashJoin : : : :- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometSubqueryBroadcast + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter @@ -30,12 +29,11 @@ CometNativeColumnarToRow : : : : +- CometBroadcastHashJoin : : : : :- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- CometSubqueryBroadcast + : : : : : +- CometBroadcastExchange + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : +- CometBroadcastExchange : : : : +- CometProject : : : : +- CometFilter @@ -60,12 +58,11 @@ CometNativeColumnarToRow : : : : +- CometBroadcastHashJoin : : : : :- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- CometSubqueryBroadcast + : : : : : +- CometBroadcastExchange + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : +- CometBroadcastExchange : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.customer @@ -97,12 +94,11 @@ CometNativeColumnarToRow : : +- CometProject : : +- CometBroadcastHashJoin : : :- CometNativeScan parquet spark_catalog.default.web_sales - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometSubqueryBroadcast + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- CometBroadcastExchange : : +- CometProject : : +- CometFilter @@ -115,12 +111,11 @@ CometNativeColumnarToRow : : : +- CometBroadcastHashJoin : : : :- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometSubqueryBroadcast + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter @@ -148,4 +143,4 @@ CometNativeColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.date_dim -Comet accelerated 130 out of 142 eligible operators (91%). Final plan contains 7 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 135 out of 142 eligible operators (95%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q23b.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q23b.native_datafusion/extended.txt index 52090c668e..79baaf1aa9 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q23b.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q23b.native_datafusion/extended.txt @@ -15,12 +15,11 @@ CometNativeColumnarToRow : : : : +- CometBroadcastHashJoin : : : : :- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- CometSubqueryBroadcast + : : : : : +- CometBroadcastExchange + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : +- CometBroadcastExchange : : : : +- CometProject : : : : +- CometFilter @@ -33,12 +32,11 @@ CometNativeColumnarToRow : : : : : +- CometBroadcastHashJoin : : : : : :- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometNativeColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : +- CometSubqueryBroadcast + : : : : : : +- CometBroadcastExchange + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : : +- CometBroadcastExchange : : : : : +- CometProject : : : : : +- CometFilter @@ -63,12 +61,11 @@ CometNativeColumnarToRow : : : : : +- CometBroadcastHashJoin : : : : : :- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometNativeColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : +- CometSubqueryBroadcast + : : : : : : +- CometBroadcastExchange + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : : +- CometBroadcastExchange : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.customer @@ -127,12 +124,11 @@ CometNativeColumnarToRow : : : +- CometBroadcastHashJoin : : : :- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometSubqueryBroadcast + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter @@ -145,12 +141,11 @@ CometNativeColumnarToRow : : : : +- CometBroadcastHashJoin : : : : :- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- CometSubqueryBroadcast + : : : : : +- CometBroadcastExchange + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : +- CometBroadcastExchange : : : : +- CometProject : : : : +- CometFilter @@ -200,4 +195,4 @@ CometNativeColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.date_dim -Comet accelerated 180 out of 194 eligible operators (92%). Final plan contains 7 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 185 out of 194 eligible operators (95%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q25.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q25.native_datafusion/extended.txt index f58c012650..2650cd8f1a 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q25.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q25.native_datafusion/extended.txt @@ -19,30 +19,27 @@ CometNativeColumnarToRow : : : : : : +- CometBroadcastHashJoin : : : : : : :- CometFilter : : : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : : : : : +- SubqueryBroadcast - : : : : : : : +- BroadcastExchange - : : : : : : : +- CometNativeColumnarToRow - : : : : : : : +- CometProject - : : : : : : : +- CometFilter - : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : : +- CometSubqueryBroadcast + : : : : : : : +- CometBroadcastExchange + : : : : : : : +- CometProject + : : : : : : : +- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : : : +- CometBroadcastExchange : : : : : : +- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.store_returns - : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometNativeColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : +- CometSubqueryBroadcast + : : : : : : +- CometBroadcastExchange + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : : +- CometBroadcastExchange : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- CometSubqueryBroadcast + : : : : : +- CometBroadcastExchange + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : +- CometBroadcastExchange : : : : +- CometProject : : : : +- CometFilter @@ -64,4 +61,4 @@ CometNativeColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.item -Comet accelerated 55 out of 61 eligible operators (90%). Final plan contains 4 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 58 out of 61 eligible operators (95%). Final plan contains 1 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q26.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q26.native_datafusion/extended.txt index 71fd3bbe6f..2f7fe7b668 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q26.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q26.native_datafusion/extended.txt @@ -13,12 +13,11 @@ CometNativeColumnarToRow : : : +- CometBroadcastHashJoin : : : :- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometSubqueryBroadcast + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter @@ -36,4 +35,4 @@ CometNativeColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.promotion -Comet accelerated 33 out of 35 eligible operators (94%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 34 out of 35 eligible operators (97%). Final plan contains 1 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q27.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q27.native_datafusion/extended.txt index 15ab28f784..df27dd568f 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q27.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q27.native_datafusion/extended.txt @@ -14,12 +14,11 @@ CometNativeColumnarToRow : : : +- CometBroadcastHashJoin : : : :- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometSubqueryBroadcast + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter @@ -37,4 +36,4 @@ CometNativeColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.item -Comet accelerated 34 out of 36 eligible operators (94%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 35 out of 36 eligible operators (97%). Final plan contains 1 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q29.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q29.native_datafusion/extended.txt index f58c012650..2650cd8f1a 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q29.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q29.native_datafusion/extended.txt @@ -19,30 +19,27 @@ CometNativeColumnarToRow : : : : : : +- CometBroadcastHashJoin : : : : : : :- CometFilter : : : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : : : : : +- SubqueryBroadcast - : : : : : : : +- BroadcastExchange - : : : : : : : +- CometNativeColumnarToRow - : : : : : : : +- CometProject - : : : : : : : +- CometFilter - : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : : +- CometSubqueryBroadcast + : : : : : : : +- CometBroadcastExchange + : : : : : : : +- CometProject + : : : : : : : +- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : : : +- CometBroadcastExchange : : : : : : +- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.store_returns - : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometNativeColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : +- CometSubqueryBroadcast + : : : : : : +- CometBroadcastExchange + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : : +- CometBroadcastExchange : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- CometSubqueryBroadcast + : : : : : +- CometBroadcastExchange + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : +- CometBroadcastExchange : : : : +- CometProject : : : : +- CometFilter @@ -64,4 +61,4 @@ CometNativeColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.item -Comet accelerated 55 out of 61 eligible operators (90%). Final plan contains 4 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 58 out of 61 eligible operators (95%). Final plan contains 1 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q30.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q30.native_datafusion/extended.txt index 18d0a12a03..2a22518df4 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q30.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q30.native_datafusion/extended.txt @@ -16,12 +16,11 @@ CometNativeColumnarToRow : : : : +- CometBroadcastHashJoin : : : : :- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.web_returns - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- CometSubqueryBroadcast + : : : : : +- CometBroadcastExchange + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : +- CometBroadcastExchange : : : : +- CometProject : : : : +- CometFilter @@ -44,12 +43,11 @@ CometNativeColumnarToRow : : : +- CometBroadcastHashJoin : : : :- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.web_returns - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometSubqueryBroadcast + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter @@ -67,4 +65,4 @@ CometNativeColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.customer_address -Comet accelerated 61 out of 65 eligible operators (93%). Final plan contains 3 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 63 out of 65 eligible operators (96%). Final plan contains 1 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q31.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q31.native_datafusion/extended.txt index d747a5243e..bf4fb2aa77 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q31.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q31.native_datafusion/extended.txt @@ -18,11 +18,10 @@ CometNativeColumnarToRow : : : : : : +- CometBroadcastHashJoin : : : : : : :- CometFilter : : : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : : : : : +- SubqueryBroadcast - : : : : : : : +- BroadcastExchange - : : : : : : : +- CometNativeColumnarToRow - : : : : : : : +- CometFilter - : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : : +- CometSubqueryBroadcast + : : : : : : : +- CometBroadcastExchange + : : : : : : : +- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : : : +- CometBroadcastExchange : : : : : : +- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -39,11 +38,10 @@ CometNativeColumnarToRow : : : : : +- CometBroadcastHashJoin : : : : : :- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometNativeColumnarToRow - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : +- CometSubqueryBroadcast + : : : : : : +- CometBroadcastExchange + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : : +- CometBroadcastExchange : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -60,11 +58,10 @@ CometNativeColumnarToRow : : : : +- CometBroadcastHashJoin : : : : :- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- CometSubqueryBroadcast + : : : : : +- CometBroadcastExchange + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : +- CometBroadcastExchange : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -81,11 +78,10 @@ CometNativeColumnarToRow : : : +- CometBroadcastHashJoin : : : :- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometSubqueryBroadcast + : : : : +- CometBroadcastExchange + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- CometBroadcastExchange : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -102,11 +98,10 @@ CometNativeColumnarToRow : : +- CometBroadcastHashJoin : : :- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometSubqueryBroadcast + : : : +- CometBroadcastExchange + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- CometBroadcastExchange : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -123,11 +118,10 @@ CometNativeColumnarToRow : +- CometBroadcastHashJoin : :- CometFilter : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometSubqueryBroadcast + : : +- CometBroadcastExchange + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim : +- CometBroadcastExchange : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -135,4 +129,4 @@ CometNativeColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.customer_address -Comet accelerated 117 out of 129 eligible operators (90%). Final plan contains 7 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 123 out of 129 eligible operators (95%). Final plan contains 1 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q32.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q32.native_datafusion/extended.txt index 5d009dea2d..db07e648bc 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q32.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q32.native_datafusion/extended.txt @@ -10,12 +10,11 @@ CometNativeColumnarToRow : : +- CometBroadcastHashJoin : : :- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometSubqueryBroadcast + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- CometBroadcastExchange : : +- CometProject : : +- CometFilter @@ -29,12 +28,11 @@ CometNativeColumnarToRow : +- CometBroadcastHashJoin : :- CometFilter : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometSubqueryBroadcast + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim : +- CometBroadcastExchange : +- CometProject : +- CometFilter @@ -44,4 +42,4 @@ CometNativeColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.date_dim -Comet accelerated 38 out of 42 eligible operators (90%). Final plan contains 3 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 40 out of 42 eligible operators (95%). Final plan contains 1 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q33.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q33.native_datafusion/extended.txt index 6afaa68137..d64ba66952 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q33.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q33.native_datafusion/extended.txt @@ -15,12 +15,11 @@ CometNativeColumnarToRow : : : +- CometBroadcastHashJoin : : : :- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometSubqueryBroadcast + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter @@ -48,12 +47,11 @@ CometNativeColumnarToRow : : : +- CometBroadcastHashJoin : : : :- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometSubqueryBroadcast + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter @@ -81,12 +79,11 @@ CometNativeColumnarToRow : : +- CometBroadcastHashJoin : : :- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometSubqueryBroadcast + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- CometBroadcastExchange : : +- CometProject : : +- CometFilter @@ -104,4 +101,4 @@ CometNativeColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.item -Comet accelerated 95 out of 101 eligible operators (94%). Final plan contains 4 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 98 out of 101 eligible operators (97%). Final plan contains 1 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q34.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q34.native_datafusion/extended.txt index 1b61be2937..4c0ef9abcd 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q34.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q34.native_datafusion/extended.txt @@ -15,12 +15,11 @@ CometNativeColumnarToRow : : : +- CometBroadcastHashJoin : : : :- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometSubqueryBroadcast + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter @@ -38,4 +37,4 @@ CometNativeColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.customer -Comet accelerated 35 out of 37 eligible operators (94%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 36 out of 37 eligible operators (97%). Final plan contains 1 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q35.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q35.native_datafusion/extended.txt index 2785c3d0b1..2e79f6f464 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q35.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q35.native_datafusion/extended.txt @@ -19,12 +19,11 @@ TakeOrderedAndProject : : : : +- CometProject : : : : +- CometBroadcastHashJoin : : : : :- CometNativeScan parquet spark_catalog.default.store_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- CometSubqueryBroadcast + : : : : : +- CometBroadcastExchange + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : +- CometBroadcastExchange : : : : +- CometProject : : : : +- CometFilter @@ -34,12 +33,11 @@ TakeOrderedAndProject : : : +- CometProject : : : +- CometBroadcastHashJoin : : : :- CometNativeScan parquet spark_catalog.default.web_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometSubqueryBroadcast + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter @@ -49,12 +47,11 @@ TakeOrderedAndProject : : +- CometProject : : +- CometBroadcastHashJoin : : :- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometSubqueryBroadcast + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- CometBroadcastExchange : : +- CometProject : : +- CometFilter @@ -70,4 +67,4 @@ TakeOrderedAndProject +- CometFilter +- CometNativeScan parquet spark_catalog.default.customer_demographics -Comet accelerated 41 out of 62 eligible operators (66%). Final plan contains 9 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 44 out of 62 eligible operators (70%). Final plan contains 6 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q36.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q36.native_datafusion/extended.txt index 05ce793d8d..9480047251 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q36.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q36.native_datafusion/extended.txt @@ -16,12 +16,11 @@ TakeOrderedAndProject : : +- CometBroadcastHashJoin : : :- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometSubqueryBroadcast + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- CometBroadcastExchange : : +- CometProject : : +- CometFilter @@ -35,4 +34,4 @@ TakeOrderedAndProject +- CometFilter +- CometNativeScan parquet spark_catalog.default.store -Comet accelerated 29 out of 34 eligible operators (85%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 30 out of 34 eligible operators (88%). Final plan contains 1 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q37.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q37.native_datafusion/extended.txt index 16e3a70d9b..a83c218727 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q37.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q37.native_datafusion/extended.txt @@ -17,12 +17,11 @@ CometNativeColumnarToRow : : +- CometProject : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.inventory - : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometSubqueryBroadcast + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim : +- CometBroadcastExchange : +- CometProject : +- CometFilter @@ -31,4 +30,4 @@ CometNativeColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.catalog_sales -Comet accelerated 28 out of 30 eligible operators (93%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 29 out of 30 eligible operators (96%). Final plan contains 1 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q38.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q38.native_datafusion/extended.txt index 29422273c5..c0314d8fb0 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q38.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q38.native_datafusion/extended.txt @@ -14,12 +14,11 @@ CometNativeColumnarToRow : : : +- CometBroadcastHashJoin : : : :- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometSubqueryBroadcast + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter @@ -38,12 +37,11 @@ CometNativeColumnarToRow : : +- CometBroadcastHashJoin : : :- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometSubqueryBroadcast + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- CometBroadcastExchange : : +- CometProject : : +- CometFilter @@ -62,12 +60,11 @@ CometNativeColumnarToRow : +- CometBroadcastHashJoin : :- CometFilter : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometSubqueryBroadcast + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim : +- CometBroadcastExchange : +- CometProject : +- CometFilter @@ -77,4 +74,4 @@ CometNativeColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.customer -Comet accelerated 68 out of 74 eligible operators (91%). Final plan contains 4 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 71 out of 74 eligible operators (95%). Final plan contains 1 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q39a.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q39a.native_datafusion/extended.txt index eba1ff4dcd..67ff950c02 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q39a.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q39a.native_datafusion/extended.txt @@ -15,12 +15,11 @@ CometNativeColumnarToRow : : : +- CometBroadcastHashJoin : : : :- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.inventory - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometSubqueryBroadcast + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- CometBroadcastExchange : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.item @@ -45,12 +44,11 @@ CometNativeColumnarToRow : : +- CometBroadcastHashJoin : : :- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.inventory - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometSubqueryBroadcast + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- CometBroadcastExchange : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.item @@ -62,4 +60,4 @@ CometNativeColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.date_dim -Comet accelerated 56 out of 60 eligible operators (93%). Final plan contains 3 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 58 out of 60 eligible operators (96%). Final plan contains 1 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q39b.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q39b.native_datafusion/extended.txt index eba1ff4dcd..67ff950c02 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q39b.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q39b.native_datafusion/extended.txt @@ -15,12 +15,11 @@ CometNativeColumnarToRow : : : +- CometBroadcastHashJoin : : : :- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.inventory - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometSubqueryBroadcast + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- CometBroadcastExchange : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.item @@ -45,12 +44,11 @@ CometNativeColumnarToRow : : +- CometBroadcastHashJoin : : :- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.inventory - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometSubqueryBroadcast + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- CometBroadcastExchange : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.item @@ -62,4 +60,4 @@ CometNativeColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.date_dim -Comet accelerated 56 out of 60 eligible operators (93%). Final plan contains 3 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 58 out of 60 eligible operators (96%). Final plan contains 1 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q4.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q4.native_datafusion/extended.txt index b2914584b4..bf61cfd81e 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q4.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q4.native_datafusion/extended.txt @@ -23,11 +23,10 @@ CometNativeColumnarToRow : : : : : : +- CometBroadcastExchange : : : : : : +- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometNativeColumnarToRow - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : +- CometSubqueryBroadcast + : : : : : : +- CometBroadcastExchange + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : : +- CometBroadcastExchange : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -45,11 +44,10 @@ CometNativeColumnarToRow : : : : : +- CometBroadcastExchange : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- CometSubqueryBroadcast + : : : : : +- CometBroadcastExchange + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : +- CometBroadcastExchange : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -68,11 +66,10 @@ CometNativeColumnarToRow : : : : +- CometBroadcastExchange : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometSubqueryBroadcast + : : : : +- CometBroadcastExchange + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- CometBroadcastExchange : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -90,11 +87,10 @@ CometNativeColumnarToRow : : : +- CometBroadcastExchange : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometSubqueryBroadcast + : : : +- CometBroadcastExchange + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- CometBroadcastExchange : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -113,11 +109,10 @@ CometNativeColumnarToRow : : +- CometBroadcastExchange : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometSubqueryBroadcast + : : +- CometBroadcastExchange + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim : +- CometBroadcastExchange : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -135,13 +130,12 @@ CometNativeColumnarToRow : +- CometBroadcastExchange : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.web_sales - : +- SubqueryBroadcast - : +- BroadcastExchange - : +- CometNativeColumnarToRow - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometSubqueryBroadcast + : +- CometBroadcastExchange + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.date_dim +- CometBroadcastExchange +- CometFilter +- CometNativeScan parquet spark_catalog.default.date_dim -Comet accelerated 126 out of 138 eligible operators (91%). Final plan contains 7 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 132 out of 138 eligible operators (95%). Final plan contains 1 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q40.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q40.native_datafusion/extended.txt index 9b00a2c6a4..1a148d7ab4 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q40.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q40.native_datafusion/extended.txt @@ -15,11 +15,10 @@ CometNativeColumnarToRow : : : : +- CometExchange : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometSubqueryBroadcast + : : : : +- CometBroadcastExchange + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- CometSort : : : +- CometExchange : : : +- CometProject @@ -37,4 +36,4 @@ CometNativeColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.date_dim -Comet accelerated 34 out of 36 eligible operators (94%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 35 out of 36 eligible operators (97%). Final plan contains 1 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q45.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q45.native_datafusion/extended.txt index 7292a238d5..e7658f2548 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q45.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q45.native_datafusion/extended.txt @@ -17,12 +17,11 @@ TakeOrderedAndProject : : : : +- CometBroadcastHashJoin : : : : :- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- CometSubqueryBroadcast + : : : : : +- CometBroadcastExchange + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : +- CometBroadcastExchange : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.customer @@ -44,4 +43,4 @@ TakeOrderedAndProject +- CometFilter +- CometNativeScan parquet spark_catalog.default.item -Comet accelerated 32 out of 41 eligible operators (78%). Final plan contains 4 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 33 out of 41 eligible operators (80%). Final plan contains 3 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q46.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q46.native_datafusion/extended.txt index dbdaaa275a..f58213e7cb 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q46.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q46.native_datafusion/extended.txt @@ -17,12 +17,11 @@ CometNativeColumnarToRow : : : : : +- CometBroadcastHashJoin : : : : : :- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometNativeColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : +- CometSubqueryBroadcast + : : : : : : +- CometBroadcastExchange + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : : +- CometBroadcastExchange : : : : : +- CometProject : : : : : +- CometFilter @@ -46,4 +45,4 @@ CometNativeColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.customer_address -Comet accelerated 43 out of 45 eligible operators (95%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 44 out of 45 eligible operators (97%). Final plan contains 1 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q47.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q47.native_datafusion/extended.txt index 0886059452..54fa1daf2e 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q47.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q47.native_datafusion/extended.txt @@ -26,11 +26,10 @@ TakeOrderedAndProject : : : : +- CometBroadcastExchange : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometSubqueryBroadcast + : : : : +- CometBroadcastExchange + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- CometBroadcastExchange : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -58,11 +57,10 @@ TakeOrderedAndProject : : : +- CometBroadcastExchange : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometSubqueryBroadcast + : : : +- CometBroadcastExchange + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- CometBroadcastExchange : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -90,11 +88,10 @@ TakeOrderedAndProject : : +- CometBroadcastExchange : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometSubqueryBroadcast + : : +- CometBroadcastExchange + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim : +- CometBroadcastExchange : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -102,4 +99,4 @@ TakeOrderedAndProject +- CometFilter +- CometNativeScan parquet spark_catalog.default.store -Comet accelerated 75 out of 97 eligible operators (77%). Final plan contains 6 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 78 out of 97 eligible operators (80%). Final plan contains 3 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q48.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q48.native_datafusion/extended.txt index 75a1c725c2..4b1541c674 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q48.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q48.native_datafusion/extended.txt @@ -12,12 +12,11 @@ CometNativeColumnarToRow : : : +- CometBroadcastHashJoin : : : :- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometSubqueryBroadcast + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- CometBroadcastExchange : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.store @@ -34,4 +33,4 @@ CometNativeColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.date_dim -Comet accelerated 31 out of 33 eligible operators (93%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 32 out of 33 eligible operators (96%). Final plan contains 1 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q49.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q49.native_datafusion/extended.txt index 6d7dadcde9..f2ae44fb86 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q49.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q49.native_datafusion/extended.txt @@ -23,12 +23,11 @@ CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometSubqueryBroadcast + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- CometProject : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.web_returns @@ -55,12 +54,11 @@ CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometSubqueryBroadcast + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- CometProject : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.catalog_returns @@ -87,12 +85,11 @@ CometNativeColumnarToRow : : +- CometProject : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometSubqueryBroadcast + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim : +- CometProject : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.store_returns @@ -101,4 +98,4 @@ CometNativeColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.date_dim -Comet accelerated 72 out of 95 eligible operators (75%). Final plan contains 7 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 75 out of 95 eligible operators (78%). Final plan contains 4 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q5.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q5.native_datafusion/extended.txt index 719229a9a3..adef074558 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q5.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q5.native_datafusion/extended.txt @@ -16,21 +16,19 @@ CometNativeColumnarToRow : : : :- CometProject : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometSubqueryBroadcast + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.store_returns - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometSubqueryBroadcast + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- CometBroadcastExchange : : +- CometProject : : +- CometFilter @@ -50,21 +48,19 @@ CometNativeColumnarToRow : : : :- CometProject : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometSubqueryBroadcast + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.catalog_returns - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometSubqueryBroadcast + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- CometBroadcastExchange : : +- CometProject : : +- CometFilter @@ -84,22 +80,20 @@ CometNativeColumnarToRow : : :- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometSubqueryBroadcast + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- CometProject : : +- CometBroadcastHashJoin : : :- CometBroadcastExchange : : : +- CometNativeScan parquet spark_catalog.default.web_returns - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometSubqueryBroadcast + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- CometProject : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.web_sales @@ -112,4 +106,4 @@ CometNativeColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.web_site -Comet accelerated 94 out of 106 eligible operators (88%). Final plan contains 7 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 100 out of 106 eligible operators (94%). Final plan contains 1 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q50.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q50.native_datafusion/extended.txt index c013381de0..e20ab4a6d6 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q50.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q50.native_datafusion/extended.txt @@ -16,12 +16,11 @@ CometNativeColumnarToRow : : : +- CometBroadcastExchange : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.store_returns - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometSubqueryBroadcast + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- CometBroadcastExchange : : +- CometProject : : +- CometFilter @@ -34,4 +33,4 @@ CometNativeColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.date_dim -Comet accelerated 31 out of 33 eligible operators (93%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 32 out of 33 eligible operators (96%). Final plan contains 1 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q51.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q51.native_datafusion/extended.txt index 425e772118..173d1f064f 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q51.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q51.native_datafusion/extended.txt @@ -20,12 +20,11 @@ TakeOrderedAndProject : +- CometBroadcastHashJoin : :- CometFilter : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometSubqueryBroadcast + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim : +- CometBroadcastExchange : +- CometProject : +- CometFilter @@ -44,15 +43,14 @@ TakeOrderedAndProject +- CometBroadcastHashJoin :- CometFilter : +- CometNativeScan parquet spark_catalog.default.store_sales - : +- SubqueryBroadcast - : +- BroadcastExchange - : +- CometNativeColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometSubqueryBroadcast + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.date_dim +- CometBroadcastExchange +- CometProject +- CometFilter +- CometNativeScan parquet spark_catalog.default.date_dim -Comet accelerated 40 out of 51 eligible operators (78%). Final plan contains 5 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 42 out of 51 eligible operators (82%). Final plan contains 3 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q53.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q53.native_datafusion/extended.txt index 936cde052f..9a8a9bbba4 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q53.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q53.native_datafusion/extended.txt @@ -20,12 +20,11 @@ TakeOrderedAndProject : : +- CometBroadcastExchange : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometSubqueryBroadcast + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim : +- CometBroadcastExchange : +- CometProject : +- CometFilter @@ -34,4 +33,4 @@ TakeOrderedAndProject +- CometFilter +- CometNativeScan parquet spark_catalog.default.store -Comet accelerated 27 out of 33 eligible operators (81%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 28 out of 33 eligible operators (84%). Final plan contains 1 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q54.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q54.native_datafusion/extended.txt index e7fd40210f..9e1691c2fc 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q54.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q54.native_datafusion/extended.txt @@ -27,21 +27,19 @@ CometNativeColumnarToRow : : : : : : : :- CometProject : : : : : : : : +- CometFilter : : : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : : : : : +- SubqueryBroadcast - : : : : : : : : +- BroadcastExchange - : : : : : : : : +- CometNativeColumnarToRow - : : : : : : : : +- CometProject - : : : : : : : : +- CometFilter - : : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : : : +- CometSubqueryBroadcast + : : : : : : : : +- CometBroadcastExchange + : : : : : : : : +- CometProject + : : : : : : : : +- CometFilter + : : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : : : : +- CometProject : : : : : : : +- CometFilter : : : : : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : : : : : +- SubqueryBroadcast - : : : : : : : +- BroadcastExchange - : : : : : : : +- CometNativeColumnarToRow - : : : : : : : +- CometProject - : : : : : : : +- CometFilter - : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : : +- CometSubqueryBroadcast + : : : : : : : +- CometBroadcastExchange + : : : : : : : +- CometProject + : : : : : : : +- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : : : +- CometBroadcastExchange : : : : : : +- CometProject : : : : : : +- CometFilter @@ -56,28 +54,27 @@ CometNativeColumnarToRow : : : +- CometBroadcastExchange : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : : :- Subquery - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometHashAggregate - : : : : : +- CometExchange - : : : : : +- CometHashAggregate - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : +- Subquery - : : : : +- CometNativeColumnarToRow - : : : : +- CometHashAggregate - : : : : +- CometExchange - : : : : +- CometHashAggregate - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometSubqueryBroadcast + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : : :- Subquery + : : : : : +- CometNativeColumnarToRow + : : : : : +- CometHashAggregate + : : : : : +- CometExchange + : : : : : +- CometHashAggregate + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- Subquery + : : : : +- CometNativeColumnarToRow + : : : : +- CometHashAggregate + : : : : +- CometExchange + : : : : +- CometHashAggregate + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- CometBroadcastExchange : : +- CometProject : : +- CometFilter @@ -89,8 +86,22 @@ CometNativeColumnarToRow +- CometBroadcastExchange +- CometProject +- CometFilter - : :- ReusedSubquery - : +- ReusedSubquery + : :- Subquery + : : +- CometNativeColumnarToRow + : : +- CometHashAggregate + : : +- CometExchange + : : +- CometHashAggregate + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- Subquery + : +- CometNativeColumnarToRow + : +- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.date_dim +- CometNativeScan parquet spark_catalog.default.date_dim -Comet accelerated 78 out of 88 eligible operators (88%). Final plan contains 6 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 93 out of 100 eligible operators (93%). Final plan contains 5 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q56.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q56.native_datafusion/extended.txt index d52c40d57b..0744bc307d 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q56.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q56.native_datafusion/extended.txt @@ -15,12 +15,11 @@ CometNativeColumnarToRow : : : +- CometBroadcastHashJoin : : : :- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometSubqueryBroadcast + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter @@ -49,12 +48,11 @@ CometNativeColumnarToRow : : : +- CometBroadcastHashJoin : : : :- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometSubqueryBroadcast + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter @@ -83,12 +81,11 @@ CometNativeColumnarToRow : : +- CometBroadcastHashJoin : : :- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometSubqueryBroadcast + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- CometBroadcastExchange : : +- CometProject : : +- CometFilter @@ -107,4 +104,4 @@ CometNativeColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.item -Comet accelerated 98 out of 104 eligible operators (94%). Final plan contains 4 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 101 out of 104 eligible operators (97%). Final plan contains 1 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q57.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q57.native_datafusion/extended.txt index 024bfeeeea..aa4ccf3246 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q57.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q57.native_datafusion/extended.txt @@ -26,11 +26,10 @@ TakeOrderedAndProject : : : : +- CometBroadcastExchange : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometSubqueryBroadcast + : : : : +- CometBroadcastExchange + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- CometBroadcastExchange : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -58,11 +57,10 @@ TakeOrderedAndProject : : : +- CometBroadcastExchange : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometSubqueryBroadcast + : : : +- CometBroadcastExchange + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- CometBroadcastExchange : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -90,11 +88,10 @@ TakeOrderedAndProject : : +- CometBroadcastExchange : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometSubqueryBroadcast + : : +- CometBroadcastExchange + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim : +- CometBroadcastExchange : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -102,4 +99,4 @@ TakeOrderedAndProject +- CometFilter +- CometNativeScan parquet spark_catalog.default.call_center -Comet accelerated 75 out of 97 eligible operators (77%). Final plan contains 6 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 78 out of 97 eligible operators (80%). Final plan contains 3 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q58.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q58.native_datafusion/extended.txt index da8f8ec939..2e5b58d22e 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q58.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q58.native_datafusion/extended.txt @@ -14,22 +14,21 @@ CometNativeColumnarToRow : : : +- CometBroadcastHashJoin : : : :- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : : +- Subquery - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometSubqueryBroadcast + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : : +- Subquery + : : : : : +- CometNativeColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter @@ -59,22 +58,21 @@ CometNativeColumnarToRow : : +- CometBroadcastHashJoin : : :- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : : +- Subquery - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometSubqueryBroadcast + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : : +- Subquery + : : : : +- CometNativeColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- CometBroadcastExchange : : +- CometProject : : +- CometFilter @@ -104,22 +102,21 @@ CometNativeColumnarToRow : +- CometBroadcastHashJoin : :- CometFilter : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : : +- Subquery - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometSubqueryBroadcast + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : : +- Subquery + : : : +- CometNativeColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometNativeScan parquet spark_catalog.default.date_dim : +- CometBroadcastExchange : +- CometProject : +- CometFilter @@ -139,4 +136,4 @@ CometNativeColumnarToRow : +- CometNativeScan parquet spark_catalog.default.date_dim +- CometNativeScan parquet spark_catalog.default.date_dim -Comet accelerated 118 out of 130 eligible operators (90%). Final plan contains 10 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 121 out of 130 eligible operators (93%). Final plan contains 7 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q6.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q6.native_datafusion/extended.txt index 1cb21cb650..4b31bf751a 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q6.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q6.native_datafusion/extended.txt @@ -21,24 +21,30 @@ CometNativeColumnarToRow : : +- CometBroadcastExchange : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometProject - : : +- CometFilter - : : : +- Subquery - : : : +- CometNativeColumnarToRow - : : : +- CometHashAggregate - : : : +- CometExchange - : : : +- CometHashAggregate - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometSubqueryBroadcast + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : : +- Subquery + : : : +- CometNativeColumnarToRow + : : : +- CometHashAggregate + : : : +- CometExchange + : : : +- CometHashAggregate + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometNativeScan parquet spark_catalog.default.date_dim : +- CometBroadcastExchange : +- CometProject : +- CometFilter - : : +- ReusedSubquery + : : +- Subquery + : : +- CometNativeColumnarToRow + : : +- CometHashAggregate + : : +- CometExchange + : : +- CometHashAggregate + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim : +- CometNativeScan parquet spark_catalog.default.date_dim +- CometBroadcastExchange +- CometProject @@ -54,4 +60,4 @@ CometNativeColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.item -Comet accelerated 48 out of 52 eligible operators (92%). Final plan contains 3 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 55 out of 58 eligible operators (94%). Final plan contains 3 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q60.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q60.native_datafusion/extended.txt index d52c40d57b..0744bc307d 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q60.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q60.native_datafusion/extended.txt @@ -15,12 +15,11 @@ CometNativeColumnarToRow : : : +- CometBroadcastHashJoin : : : :- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometSubqueryBroadcast + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter @@ -49,12 +48,11 @@ CometNativeColumnarToRow : : : +- CometBroadcastHashJoin : : : :- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometSubqueryBroadcast + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter @@ -83,12 +81,11 @@ CometNativeColumnarToRow : : +- CometBroadcastHashJoin : : :- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometSubqueryBroadcast + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- CometBroadcastExchange : : +- CometProject : : +- CometFilter @@ -107,4 +104,4 @@ CometNativeColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.item -Comet accelerated 98 out of 104 eligible operators (94%). Final plan contains 4 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 101 out of 104 eligible operators (97%). Final plan contains 1 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q61.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q61.native_datafusion/extended.txt index ef8907c7d8..8d91cbe7a2 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q61.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q61.native_datafusion/extended.txt @@ -18,12 +18,11 @@ Project : : : : : : +- CometBroadcastHashJoin : : : : : : :- CometFilter : : : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : : : : : +- SubqueryBroadcast - : : : : : : : +- BroadcastExchange - : : : : : : : +- CometNativeColumnarToRow - : : : : : : : +- CometProject - : : : : : : : +- CometFilter - : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : : +- CometSubqueryBroadcast + : : : : : : : +- CometBroadcastExchange + : : : : : : : +- CometProject + : : : : : : : +- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : : : +- CometBroadcastExchange : : : : : : +- CometProject : : : : : : +- CometFilter @@ -64,12 +63,11 @@ Project : : : : +- CometBroadcastHashJoin : : : : :- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- CometSubqueryBroadcast + : : : : : +- CometBroadcastExchange + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : +- CometBroadcastExchange : : : : +- CometProject : : : : +- CometFilter @@ -90,4 +88,4 @@ Project +- CometFilter +- CometNativeScan parquet spark_catalog.default.item -Comet accelerated 80 out of 87 eligible operators (91%). Final plan contains 4 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 82 out of 87 eligible operators (94%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q63.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q63.native_datafusion/extended.txt index 936cde052f..9a8a9bbba4 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q63.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q63.native_datafusion/extended.txt @@ -20,12 +20,11 @@ TakeOrderedAndProject : : +- CometBroadcastExchange : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometSubqueryBroadcast + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim : +- CometBroadcastExchange : +- CometProject : +- CometFilter @@ -34,4 +33,4 @@ TakeOrderedAndProject +- CometFilter +- CometNativeScan parquet spark_catalog.default.store -Comet accelerated 27 out of 33 eligible operators (81%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 28 out of 33 eligible operators (84%). Final plan contains 1 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q64.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q64.native_datafusion/extended.txt index f1fe6fb3e3..14ffe4eab9 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q64.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q64.native_datafusion/extended.txt @@ -46,11 +46,10 @@ CometNativeColumnarToRow : : : : : : : : : : : : : : : : : :- CometBroadcastExchange : : : : : : : : : : : : : : : : : : +- CometFilter : : : : : : : : : : : : : : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : : : : : : : : : : : : : : : : +- SubqueryBroadcast - : : : : : : : : : : : : : : : : : : +- BroadcastExchange - : : : : : : : : : : : : : : : : : : +- CometNativeColumnarToRow - : : : : : : : : : : : : : : : : : : +- CometFilter - : : : : : : : : : : : : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : : : : : : : : : : : : : +- CometSubqueryBroadcast + : : : : : : : : : : : : : : : : : : +- CometBroadcastExchange + : : : : : : : : : : : : : : : : : : +- CometFilter + : : : : : : : : : : : : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : : : : : : : : : : : : : : +- CometProject : : : : : : : : : : : : : : : : : +- CometFilter : : : : : : : : : : : : : : : : : +- CometNativeScan parquet spark_catalog.default.store_returns @@ -166,11 +165,10 @@ CometNativeColumnarToRow : : : : : : : : : : : : : : : : :- CometBroadcastExchange : : : : : : : : : : : : : : : : : +- CometFilter : : : : : : : : : : : : : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : : : : : : : : : : : : : : : +- SubqueryBroadcast - : : : : : : : : : : : : : : : : : +- BroadcastExchange - : : : : : : : : : : : : : : : : : +- CometNativeColumnarToRow - : : : : : : : : : : : : : : : : : +- CometFilter - : : : : : : : : : : : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : : : : : : : : : : : : +- CometSubqueryBroadcast + : : : : : : : : : : : : : : : : : +- CometBroadcastExchange + : : : : : : : : : : : : : : : : : +- CometFilter + : : : : : : : : : : : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : : : : : : : : : : : : : +- CometProject : : : : : : : : : : : : : : : : +- CometFilter : : : : : : : : : : : : : : : : +- CometNativeScan parquet spark_catalog.default.store_returns @@ -244,4 +242,4 @@ CometNativeColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.item -Comet accelerated 238 out of 242 eligible operators (98%). Final plan contains 3 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 240 out of 242 eligible operators (99%). Final plan contains 1 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q65.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q65.native_datafusion/extended.txt index 167a3386c2..21d60497a1 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q65.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q65.native_datafusion/extended.txt @@ -17,12 +17,11 @@ CometNativeColumnarToRow : : +- CometBroadcastHashJoin : : :- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometSubqueryBroadcast + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- CometBroadcastExchange : : +- CometProject : : +- CometFilter @@ -43,15 +42,14 @@ CometNativeColumnarToRow +- CometBroadcastHashJoin :- CometFilter : +- CometNativeScan parquet spark_catalog.default.store_sales - : +- SubqueryBroadcast - : +- BroadcastExchange - : +- CometNativeColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometSubqueryBroadcast + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.date_dim +- CometBroadcastExchange +- CometProject +- CometFilter +- CometNativeScan parquet spark_catalog.default.date_dim -Comet accelerated 48 out of 52 eligible operators (92%). Final plan contains 3 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 50 out of 52 eligible operators (96%). Final plan contains 1 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q66.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q66.native_datafusion/extended.txt index a7423e3a59..c07ecfe46a 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q66.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q66.native_datafusion/extended.txt @@ -17,11 +17,10 @@ CometNativeColumnarToRow : : : : +- CometBroadcastHashJoin : : : : :- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- CometSubqueryBroadcast + : : : : : +- CometBroadcastExchange + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : +- CometBroadcastExchange : : : : +- CometProject : : : : +- CometFilter @@ -50,11 +49,10 @@ CometNativeColumnarToRow : : : +- CometBroadcastHashJoin : : : :- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometSubqueryBroadcast + : : : : +- CometBroadcastExchange + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter @@ -71,4 +69,4 @@ CometNativeColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.ship_mode -Comet accelerated 65 out of 69 eligible operators (94%). Final plan contains 3 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 67 out of 69 eligible operators (97%). Final plan contains 1 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q67.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q67.native_datafusion/extended.txt index 94c6cc711d..3c1bc08e97 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q67.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q67.native_datafusion/extended.txt @@ -20,12 +20,11 @@ TakeOrderedAndProject : : +- CometBroadcastHashJoin : : :- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometSubqueryBroadcast + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- CometBroadcastExchange : : +- CometProject : : +- CometFilter @@ -39,4 +38,4 @@ TakeOrderedAndProject +- CometFilter +- CometNativeScan parquet spark_catalog.default.item -Comet accelerated 30 out of 37 eligible operators (81%). Final plan contains 3 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 31 out of 37 eligible operators (83%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q68.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q68.native_datafusion/extended.txt index dbdaaa275a..f58213e7cb 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q68.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q68.native_datafusion/extended.txt @@ -17,12 +17,11 @@ CometNativeColumnarToRow : : : : : +- CometBroadcastHashJoin : : : : : :- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometNativeColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : +- CometSubqueryBroadcast + : : : : : : +- CometBroadcastExchange + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : : +- CometBroadcastExchange : : : : : +- CometProject : : : : : +- CometFilter @@ -46,4 +45,4 @@ CometNativeColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.customer_address -Comet accelerated 43 out of 45 eligible operators (95%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 44 out of 45 eligible operators (97%). Final plan contains 1 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q69.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q69.native_datafusion/extended.txt index 3e0ba48add..6d398580fb 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q69.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q69.native_datafusion/extended.txt @@ -18,12 +18,11 @@ TakeOrderedAndProject : : : : +- CometProject : : : : +- CometBroadcastHashJoin : : : : :- CometNativeScan parquet spark_catalog.default.store_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- CometSubqueryBroadcast + : : : : : +- CometBroadcastExchange + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : +- CometBroadcastExchange : : : : +- CometProject : : : : +- CometFilter @@ -33,12 +32,11 @@ TakeOrderedAndProject : : : +- CometProject : : : +- CometBroadcastHashJoin : : : :- CometNativeScan parquet spark_catalog.default.web_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometSubqueryBroadcast + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter @@ -48,12 +46,11 @@ TakeOrderedAndProject : : +- CometProject : : +- CometBroadcastHashJoin : : :- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometSubqueryBroadcast + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- CometBroadcastExchange : : +- CometProject : : +- CometFilter @@ -69,4 +66,4 @@ TakeOrderedAndProject +- CometFilter +- CometNativeScan parquet spark_catalog.default.customer_demographics -Comet accelerated 41 out of 61 eligible operators (67%). Final plan contains 9 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 44 out of 61 eligible operators (72%). Final plan contains 6 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q7.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q7.native_datafusion/extended.txt index 42206d1be5..8750081c55 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q7.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q7.native_datafusion/extended.txt @@ -13,12 +13,11 @@ CometNativeColumnarToRow : : : +- CometBroadcastHashJoin : : : :- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometSubqueryBroadcast + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter @@ -36,4 +35,4 @@ CometNativeColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.promotion -Comet accelerated 33 out of 35 eligible operators (94%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 34 out of 35 eligible operators (97%). Final plan contains 1 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q70.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q70.native_datafusion/extended.txt index befbf52ae5..d1ff1b903e 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q70.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q70.native_datafusion/extended.txt @@ -16,12 +16,11 @@ TakeOrderedAndProject : +- CometBroadcastHashJoin : :- CometFilter : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometSubqueryBroadcast + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim : +- CometBroadcastExchange : +- CometProject : +- CometFilter @@ -48,12 +47,11 @@ TakeOrderedAndProject : +- CometBroadcastHashJoin : :- CometFilter : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometSubqueryBroadcast + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim : +- CometBroadcastExchange : +- CometProject : +- CometFilter @@ -63,4 +61,4 @@ TakeOrderedAndProject +- CometFilter +- CometNativeScan parquet spark_catalog.default.date_dim -Comet accelerated 37 out of 57 eligible operators (64%). Final plan contains 7 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 39 out of 57 eligible operators (68%). Final plan contains 5 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q71.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q71.native_datafusion/extended.txt index 7f76dd192d..834098d4ee 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q71.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q71.native_datafusion/extended.txt @@ -17,12 +17,11 @@ CometNativeColumnarToRow : : +- CometBroadcastHashJoin : : :- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometSubqueryBroadcast + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- CometBroadcastExchange : : +- CometProject : : +- CometFilter @@ -31,12 +30,11 @@ CometNativeColumnarToRow : : +- CometBroadcastHashJoin : : :- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometSubqueryBroadcast + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- CometBroadcastExchange : : +- CometProject : : +- CometFilter @@ -45,12 +43,11 @@ CometNativeColumnarToRow : +- CometBroadcastHashJoin : :- CometFilter : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometSubqueryBroadcast + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim : +- CometBroadcastExchange : +- CometProject : +- CometFilter @@ -60,4 +57,4 @@ CometNativeColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.time_dim -Comet accelerated 51 out of 57 eligible operators (89%). Final plan contains 4 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 54 out of 57 eligible operators (94%). Final plan contains 1 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q72.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q72.native_datafusion/extended.txt index 623a68ddab..d443b31d1e 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q72.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q72.native_datafusion/extended.txt @@ -27,12 +27,11 @@ CometNativeColumnarToRow : : : : : : : : : +- CometBroadcastHashJoin : : : : : : : : : :- CometFilter : : : : : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : : : : : : : +- SubqueryBroadcast - : : : : : : : : : : +- BroadcastExchange - : : : : : : : : : : +- CometNativeColumnarToRow - : : : : : : : : : : +- CometProject - : : : : : : : : : : +- CometFilter - : : : : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : : : : : +- CometSubqueryBroadcast + : : : : : : : : : : +- CometBroadcastExchange + : : : : : : : : : : +- CometProject + : : : : : : : : : : +- CometFilter + : : : : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : : : : : : +- CometBroadcastExchange : : : : : : : : : +- CometFilter : : : : : : : : : +- CometNativeScan parquet spark_catalog.default.inventory @@ -69,4 +68,4 @@ CometNativeColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.catalog_returns -Comet accelerated 66 out of 68 eligible operators (97%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 67 out of 68 eligible operators (98%). Final plan contains 1 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q73.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q73.native_datafusion/extended.txt index 1b61be2937..4c0ef9abcd 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q73.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q73.native_datafusion/extended.txt @@ -15,12 +15,11 @@ CometNativeColumnarToRow : : : +- CometBroadcastHashJoin : : : :- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometSubqueryBroadcast + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter @@ -38,4 +37,4 @@ CometNativeColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.customer -Comet accelerated 35 out of 37 eligible operators (94%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 36 out of 37 eligible operators (97%). Final plan contains 1 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q74.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q74.native_datafusion/extended.txt index 51a3f5ca94..7768138645 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q74.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q74.native_datafusion/extended.txt @@ -19,11 +19,10 @@ CometNativeColumnarToRow : : : : +- CometBroadcastExchange : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometSubqueryBroadcast + : : : : +- CometBroadcastExchange + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- CometBroadcastExchange : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -41,11 +40,10 @@ CometNativeColumnarToRow : : : +- CometBroadcastExchange : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometSubqueryBroadcast + : : : +- CometBroadcastExchange + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- CometBroadcastExchange : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -64,11 +62,10 @@ CometNativeColumnarToRow : : +- CometBroadcastExchange : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometSubqueryBroadcast + : : +- CometBroadcastExchange + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim : +- CometBroadcastExchange : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -86,13 +83,12 @@ CometNativeColumnarToRow : +- CometBroadcastExchange : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.web_sales - : +- SubqueryBroadcast - : +- BroadcastExchange - : +- CometNativeColumnarToRow - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometSubqueryBroadcast + : +- CometBroadcastExchange + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.date_dim +- CometBroadcastExchange +- CometFilter +- CometNativeScan parquet spark_catalog.default.date_dim -Comet accelerated 83 out of 91 eligible operators (91%). Final plan contains 5 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 87 out of 91 eligible operators (95%). Final plan contains 1 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q75.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q75.native_datafusion/extended.txt index a943308f1e..ee7c0af216 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q75.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q75.native_datafusion/extended.txt @@ -22,11 +22,10 @@ CometNativeColumnarToRow : : : : +- CometBroadcastHashJoin : : : : :- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- CometSubqueryBroadcast + : : : : : +- CometBroadcastExchange + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : +- CometBroadcastExchange : : : : +- CometProject : : : : +- CometFilter @@ -49,11 +48,10 @@ CometNativeColumnarToRow : : : : +- CometBroadcastHashJoin : : : : :- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- CometSubqueryBroadcast + : : : : : +- CometBroadcastExchange + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : +- CometBroadcastExchange : : : : +- CometProject : : : : +- CometFilter @@ -76,11 +74,10 @@ CometNativeColumnarToRow : : : +- CometBroadcastHashJoin : : : :- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometSubqueryBroadcast + : : : : +- CometBroadcastExchange + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter @@ -113,11 +110,10 @@ CometNativeColumnarToRow : : : +- CometBroadcastHashJoin : : : :- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometSubqueryBroadcast + : : : : +- CometBroadcastExchange + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter @@ -140,11 +136,10 @@ CometNativeColumnarToRow : : : +- CometBroadcastHashJoin : : : :- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometSubqueryBroadcast + : : : : +- CometBroadcastExchange + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter @@ -167,11 +162,10 @@ CometNativeColumnarToRow : : +- CometBroadcastHashJoin : : :- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometSubqueryBroadcast + : : : +- CometBroadcastExchange + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- CometBroadcastExchange : : +- CometProject : : +- CometFilter @@ -185,4 +179,4 @@ CometNativeColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.web_returns -Comet accelerated 167 out of 179 eligible operators (93%). Final plan contains 7 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 173 out of 179 eligible operators (96%). Final plan contains 1 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q77.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q77.native_datafusion/extended.txt index 835f7a449a..8805a2e6c2 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q77.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q77.native_datafusion/extended.txt @@ -17,12 +17,11 @@ TakeOrderedAndProject : : : +- CometBroadcastHashJoin : : : :- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometSubqueryBroadcast + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter @@ -40,12 +39,11 @@ TakeOrderedAndProject : : +- CometBroadcastHashJoin : : :- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.store_returns - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometSubqueryBroadcast + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- CometBroadcastExchange : : +- CometProject : : +- CometFilter @@ -63,12 +61,11 @@ TakeOrderedAndProject : : +- CometProject : : +- CometBroadcastHashJoin : : :- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometSubqueryBroadcast + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- CometBroadcastExchange : : +- CometProject : : +- CometFilter @@ -80,12 +77,11 @@ TakeOrderedAndProject : +- CometProject : +- CometBroadcastHashJoin : :- CometNativeScan parquet spark_catalog.default.catalog_returns - : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometSubqueryBroadcast + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim : +- CometBroadcastExchange : +- CometProject : +- CometFilter @@ -102,12 +98,11 @@ TakeOrderedAndProject : : +- CometBroadcastHashJoin : : :- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometSubqueryBroadcast + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- CometBroadcastExchange : : +- CometProject : : +- CometFilter @@ -125,12 +120,11 @@ TakeOrderedAndProject : +- CometBroadcastHashJoin : :- CometFilter : : +- CometNativeScan parquet spark_catalog.default.web_returns - : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometSubqueryBroadcast + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim : +- CometBroadcastExchange : +- CometProject : +- CometFilter @@ -139,4 +133,4 @@ TakeOrderedAndProject +- CometFilter +- CometNativeScan parquet spark_catalog.default.web_page -Comet accelerated 109 out of 129 eligible operators (84%). Final plan contains 11 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 115 out of 129 eligible operators (89%). Final plan contains 5 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q78.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q78.native_datafusion/extended.txt index ebfdcc3c62..8426b9653c 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q78.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q78.native_datafusion/extended.txt @@ -17,11 +17,10 @@ TakeOrderedAndProject : : : : +- CometExchange : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometSubqueryBroadcast + : : : : +- CometBroadcastExchange + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- CometSort : : : +- CometExchange : : : +- CometProject @@ -44,11 +43,10 @@ TakeOrderedAndProject : : : +- CometExchange : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometSubqueryBroadcast + : : : +- CometBroadcastExchange + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- CometSort : : +- CometExchange : : +- CometProject @@ -71,11 +69,10 @@ TakeOrderedAndProject : : +- CometExchange : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometSubqueryBroadcast + : : +- CometBroadcastExchange + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim : +- CometSort : +- CometExchange : +- CometProject @@ -85,4 +82,4 @@ TakeOrderedAndProject +- CometFilter +- CometNativeScan parquet spark_catalog.default.date_dim -Comet accelerated 74 out of 82 eligible operators (90%). Final plan contains 4 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 77 out of 82 eligible operators (93%). Final plan contains 1 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q79.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q79.native_datafusion/extended.txt index 526b86f475..5af6449428 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q79.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q79.native_datafusion/extended.txt @@ -13,12 +13,11 @@ CometNativeColumnarToRow : : : +- CometBroadcastHashJoin : : : :- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometSubqueryBroadcast + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter @@ -36,4 +35,4 @@ CometNativeColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.customer -Comet accelerated 33 out of 35 eligible operators (94%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 34 out of 35 eligible operators (97%). Final plan contains 1 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q8.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q8.native_datafusion/extended.txt index cb04fc12c8..209a4e2bc0 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q8.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q8.native_datafusion/extended.txt @@ -11,12 +11,11 @@ CometNativeColumnarToRow : : +- CometBroadcastHashJoin : : :- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometSubqueryBroadcast + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- CometBroadcastExchange : : +- CometProject : : +- CometFilter @@ -49,4 +48,4 @@ CometNativeColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.customer -Comet accelerated 46 out of 48 eligible operators (95%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 47 out of 48 eligible operators (97%). Final plan contains 1 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q80.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q80.native_datafusion/extended.txt index 003be45ee5..27e32a1602 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q80.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q80.native_datafusion/extended.txt @@ -22,12 +22,11 @@ CometNativeColumnarToRow : : : : : : +- CometExchange : : : : : : +- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometNativeColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : +- CometSubqueryBroadcast + : : : : : : +- CometBroadcastExchange + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : : +- CometSort : : : : : +- CometExchange : : : : : +- CometProject @@ -66,12 +65,11 @@ CometNativeColumnarToRow : : : : : : +- CometExchange : : : : : : +- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometNativeColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : +- CometSubqueryBroadcast + : : : : : : +- CometBroadcastExchange + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : : +- CometSort : : : : : +- CometExchange : : : : : +- CometProject @@ -110,12 +108,11 @@ CometNativeColumnarToRow : : : : : +- CometExchange : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- CometSubqueryBroadcast + : : : : : +- CometBroadcastExchange + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : +- CometSort : : : : +- CometExchange : : : : +- CometProject @@ -138,4 +135,4 @@ CometNativeColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.promotion -Comet accelerated 129 out of 135 eligible operators (95%). Final plan contains 4 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 132 out of 135 eligible operators (97%). Final plan contains 1 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q81.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q81.native_datafusion/extended.txt index 949d50a867..6d9aa9697d 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q81.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q81.native_datafusion/extended.txt @@ -16,12 +16,11 @@ CometNativeColumnarToRow : : : : +- CometBroadcastHashJoin : : : : :- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_returns - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- CometSubqueryBroadcast + : : : : : +- CometBroadcastExchange + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : +- CometBroadcastExchange : : : : +- CometProject : : : : +- CometFilter @@ -44,12 +43,11 @@ CometNativeColumnarToRow : : : +- CometBroadcastHashJoin : : : :- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.catalog_returns - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometSubqueryBroadcast + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter @@ -67,4 +65,4 @@ CometNativeColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.customer_address -Comet accelerated 61 out of 65 eligible operators (93%). Final plan contains 3 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 63 out of 65 eligible operators (96%). Final plan contains 1 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q82.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q82.native_datafusion/extended.txt index 30fca82ec8..e7b75e3a55 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q82.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q82.native_datafusion/extended.txt @@ -17,12 +17,11 @@ CometNativeColumnarToRow : : +- CometProject : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.inventory - : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometSubqueryBroadcast + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim : +- CometBroadcastExchange : +- CometProject : +- CometFilter @@ -31,4 +30,4 @@ CometNativeColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.store_sales -Comet accelerated 28 out of 30 eligible operators (93%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 29 out of 30 eligible operators (96%). Final plan contains 1 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q83.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q83.native_datafusion/extended.txt index 3a16430014..e4d1992492 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q83.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q83.native_datafusion/extended.txt @@ -13,21 +13,20 @@ CometNativeColumnarToRow : : : +- CometBroadcastHashJoin : : : :- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.store_returns - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometNativeScan parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometSubqueryBroadcast + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter @@ -55,21 +54,20 @@ CometNativeColumnarToRow : : +- CometBroadcastHashJoin : : :- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.catalog_returns - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometSubqueryBroadcast + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- CometBroadcastExchange : : +- CometProject : : +- CometFilter @@ -97,21 +95,20 @@ CometNativeColumnarToRow : +- CometBroadcastHashJoin : :- CometFilter : : +- CometNativeScan parquet spark_catalog.default.web_returns - : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometNativeScan parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometSubqueryBroadcast + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim : +- CometBroadcastExchange : +- CometProject : +- CometFilter @@ -130,4 +127,4 @@ CometNativeColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.date_dim -Comet accelerated 121 out of 127 eligible operators (95%). Final plan contains 4 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 124 out of 127 eligible operators (97%). Final plan contains 1 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q85.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q85.native_datafusion/extended.txt index 06537a5eec..13506e196f 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q85.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q85.native_datafusion/extended.txt @@ -20,12 +20,11 @@ CometNativeColumnarToRow : : : : : : :- CometBroadcastExchange : : : : : : : +- CometFilter : : : : : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : : : : : +- SubqueryBroadcast - : : : : : : : +- BroadcastExchange - : : : : : : : +- CometNativeColumnarToRow - : : : : : : : +- CometProject - : : : : : : : +- CometFilter - : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : : +- CometSubqueryBroadcast + : : : : : : : +- CometBroadcastExchange + : : : : : : : +- CometProject + : : : : : : : +- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : : : +- CometProject : : : : : : +- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.web_returns @@ -53,4 +52,4 @@ CometNativeColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.reason -Comet accelerated 50 out of 52 eligible operators (96%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 51 out of 52 eligible operators (98%). Final plan contains 1 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q86.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q86.native_datafusion/extended.txt index c9117e2ae0..bddf80b4d0 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q86.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q86.native_datafusion/extended.txt @@ -14,12 +14,11 @@ TakeOrderedAndProject : +- CometBroadcastHashJoin : :- CometFilter : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometSubqueryBroadcast + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim : +- CometBroadcastExchange : +- CometProject : +- CometFilter @@ -29,4 +28,4 @@ TakeOrderedAndProject +- CometFilter +- CometNativeScan parquet spark_catalog.default.item -Comet accelerated 23 out of 28 eligible operators (82%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 24 out of 28 eligible operators (85%). Final plan contains 1 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q87.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q87.native_datafusion/extended.txt index 6676ec7a4b..5c80165c1f 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q87.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q87.native_datafusion/extended.txt @@ -15,12 +15,11 @@ HashAggregate : : : +- CometBroadcastHashJoin : : : :- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometSubqueryBroadcast + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter @@ -40,12 +39,11 @@ HashAggregate : : +- CometBroadcastHashJoin : : :- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometSubqueryBroadcast + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- CometBroadcastExchange : : +- CometProject : : +- CometFilter @@ -65,12 +63,11 @@ HashAggregate : +- CometBroadcastHashJoin : :- CometFilter : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometSubqueryBroadcast + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim : +- CometBroadcastExchange : +- CometProject : +- CometFilter @@ -80,4 +77,4 @@ HashAggregate +- CometFilter +- CometNativeScan parquet spark_catalog.default.customer -Comet accelerated 61 out of 74 eligible operators (82%). Final plan contains 7 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 64 out of 74 eligible operators (86%). Final plan contains 4 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q89.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q89.native_datafusion/extended.txt index 936cde052f..9a8a9bbba4 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q89.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q89.native_datafusion/extended.txt @@ -20,12 +20,11 @@ TakeOrderedAndProject : : +- CometBroadcastExchange : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometSubqueryBroadcast + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim : +- CometBroadcastExchange : +- CometProject : +- CometFilter @@ -34,4 +33,4 @@ TakeOrderedAndProject +- CometFilter +- CometNativeScan parquet spark_catalog.default.store -Comet accelerated 27 out of 33 eligible operators (81%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 28 out of 33 eligible operators (84%). Final plan contains 1 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q91.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q91.native_datafusion/extended.txt index 28bc6eab2f..9b191f8492 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q91.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q91.native_datafusion/extended.txt @@ -22,12 +22,11 @@ CometNativeColumnarToRow : : : : : +- CometBroadcastExchange : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_returns - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- CometSubqueryBroadcast + : : : : : +- CometBroadcastExchange + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : +- CometBroadcastExchange : : : : +- CometProject : : : : +- CometFilter @@ -48,4 +47,4 @@ CometNativeColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.household_demographics -Comet accelerated 45 out of 47 eligible operators (95%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 46 out of 47 eligible operators (97%). Final plan contains 1 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q92.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q92.native_datafusion/extended.txt index 6c5b57b3a1..40926fafac 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q92.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q92.native_datafusion/extended.txt @@ -10,12 +10,11 @@ CometNativeColumnarToRow : : +- CometBroadcastHashJoin : : :- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometSubqueryBroadcast + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- CometBroadcastExchange : : +- CometProject : : +- CometFilter @@ -29,12 +28,11 @@ CometNativeColumnarToRow : +- CometBroadcastHashJoin : :- CometFilter : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometSubqueryBroadcast + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim : +- CometBroadcastExchange : +- CometProject : +- CometFilter @@ -44,4 +42,4 @@ CometNativeColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.date_dim -Comet accelerated 38 out of 42 eligible operators (90%). Final plan contains 3 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 40 out of 42 eligible operators (95%). Final plan contains 1 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q97.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q97.native_datafusion/extended.txt index b63930a880..74163dff2b 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q97.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q97.native_datafusion/extended.txt @@ -11,12 +11,11 @@ CometNativeColumnarToRow : +- CometProject : +- CometBroadcastHashJoin : :- CometNativeScan parquet spark_catalog.default.store_sales - : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometSubqueryBroadcast + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim : +- CometBroadcastExchange : +- CometProject : +- CometFilter @@ -28,15 +27,14 @@ CometNativeColumnarToRow +- CometProject +- CometBroadcastHashJoin :- CometNativeScan parquet spark_catalog.default.catalog_sales - : +- SubqueryBroadcast - : +- BroadcastExchange - : +- CometNativeColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometSubqueryBroadcast + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.date_dim +- CometBroadcastExchange +- CometProject +- CometFilter +- CometNativeScan parquet spark_catalog.default.date_dim -Comet accelerated 33 out of 37 eligible operators (89%). Final plan contains 3 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 35 out of 37 eligible operators (94%). Final plan contains 1 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q98.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q98.native_datafusion/extended.txt index 6cc173560b..f1fc5ff9a3 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q98.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q98.native_datafusion/extended.txt @@ -16,12 +16,11 @@ CometNativeColumnarToRow : +- CometBroadcastHashJoin : :- CometFilter : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometSubqueryBroadcast + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim : +- CometBroadcastExchange : +- CometProject : +- CometFilter @@ -31,4 +30,4 @@ CometNativeColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.date_dim -Comet accelerated 25 out of 29 eligible operators (86%). Final plan contains 3 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 26 out of 29 eligible operators (89%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q1.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q1.native_datafusion/extended.txt index 935c103614..5875fa3a5e 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q1.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q1.native_datafusion/extended.txt @@ -14,12 +14,11 @@ CometNativeColumnarToRow : : : +- CometBroadcastHashJoin : : : :- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.store_returns - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometSubqueryBroadcast + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter @@ -36,12 +35,11 @@ CometNativeColumnarToRow : : +- CometBroadcastHashJoin : : :- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.store_returns - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometSubqueryBroadcast + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- CometBroadcastExchange : : +- CometProject : : +- CometFilter @@ -55,4 +53,4 @@ CometNativeColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.customer -Comet accelerated 49 out of 53 eligible operators (92%). Final plan contains 3 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 51 out of 53 eligible operators (96%). Final plan contains 1 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q10.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q10.native_datafusion/extended.txt index 2785c3d0b1..2e79f6f464 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q10.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q10.native_datafusion/extended.txt @@ -19,12 +19,11 @@ TakeOrderedAndProject : : : : +- CometProject : : : : +- CometBroadcastHashJoin : : : : :- CometNativeScan parquet spark_catalog.default.store_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- CometSubqueryBroadcast + : : : : : +- CometBroadcastExchange + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : +- CometBroadcastExchange : : : : +- CometProject : : : : +- CometFilter @@ -34,12 +33,11 @@ TakeOrderedAndProject : : : +- CometProject : : : +- CometBroadcastHashJoin : : : :- CometNativeScan parquet spark_catalog.default.web_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometSubqueryBroadcast + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter @@ -49,12 +47,11 @@ TakeOrderedAndProject : : +- CometProject : : +- CometBroadcastHashJoin : : :- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometSubqueryBroadcast + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- CometBroadcastExchange : : +- CometProject : : +- CometFilter @@ -70,4 +67,4 @@ TakeOrderedAndProject +- CometFilter +- CometNativeScan parquet spark_catalog.default.customer_demographics -Comet accelerated 41 out of 62 eligible operators (66%). Final plan contains 9 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 44 out of 62 eligible operators (70%). Final plan contains 6 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q11.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q11.native_datafusion/extended.txt index 6b081dd35b..edda694b73 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q11.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q11.native_datafusion/extended.txt @@ -20,11 +20,10 @@ CometNativeColumnarToRow : : : : +- CometBroadcastExchange : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometSubqueryBroadcast + : : : : +- CometBroadcastExchange + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- CometBroadcastExchange : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -42,11 +41,10 @@ CometNativeColumnarToRow : : : +- CometBroadcastExchange : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometSubqueryBroadcast + : : : +- CometBroadcastExchange + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- CometBroadcastExchange : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -65,11 +63,10 @@ CometNativeColumnarToRow : : +- CometBroadcastExchange : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometSubqueryBroadcast + : : +- CometBroadcastExchange + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim : +- CometBroadcastExchange : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -87,13 +84,12 @@ CometNativeColumnarToRow : +- CometBroadcastExchange : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.web_sales - : +- SubqueryBroadcast - : +- BroadcastExchange - : +- CometNativeColumnarToRow - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometSubqueryBroadcast + : +- CometBroadcastExchange + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.date_dim +- CometBroadcastExchange +- CometFilter +- CometNativeScan parquet spark_catalog.default.date_dim -Comet accelerated 84 out of 92 eligible operators (91%). Final plan contains 5 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 88 out of 92 eligible operators (95%). Final plan contains 1 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q12.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q12.native_datafusion/extended.txt index a8c2531a38..a6cc526735 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q12.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q12.native_datafusion/extended.txt @@ -13,12 +13,11 @@ TakeOrderedAndProject : +- CometBroadcastHashJoin : :- CometFilter : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometSubqueryBroadcast + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim : +- CometBroadcastExchange : +- CometProject : +- CometFilter @@ -28,4 +27,4 @@ TakeOrderedAndProject +- CometFilter +- CometNativeScan parquet spark_catalog.default.date_dim -Comet accelerated 22 out of 27 eligible operators (81%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 23 out of 27 eligible operators (85%). Final plan contains 1 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q13.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q13.native_datafusion/extended.txt index 42ea1c4966..33c2dd1bbc 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q13.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q13.native_datafusion/extended.txt @@ -14,12 +14,11 @@ CometNativeColumnarToRow : : : : +- CometBroadcastHashJoin : : : : :- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- CometSubqueryBroadcast + : : : : : +- CometBroadcastExchange + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : +- CometBroadcastExchange : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.store @@ -39,4 +38,4 @@ CometNativeColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.household_demographics -Comet accelerated 36 out of 38 eligible operators (94%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 37 out of 38 eligible operators (97%). Final plan contains 1 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q14a.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q14a.native_datafusion/extended.txt index 0dcd47f57f..d2a44e9c52 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q14a.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q14a.native_datafusion/extended.txt @@ -16,12 +16,11 @@ CometNativeColumnarToRow : : :- CometProject : : : +- CometBroadcastHashJoin : : : :- CometNativeScan parquet spark_catalog.default.store_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometSubqueryBroadcast + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter @@ -29,12 +28,11 @@ CometNativeColumnarToRow : : :- CometProject : : : +- CometBroadcastHashJoin : : : :- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometSubqueryBroadcast + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter @@ -42,12 +40,11 @@ CometNativeColumnarToRow : : +- CometProject : : +- CometBroadcastHashJoin : : :- CometNativeScan parquet spark_catalog.default.web_sales - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometSubqueryBroadcast + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- CometBroadcastExchange : : +- CometProject : : +- CometFilter @@ -62,12 +59,11 @@ CometNativeColumnarToRow : : :- CometBroadcastHashJoin : : : :- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometSubqueryBroadcast + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometBroadcastHashJoin @@ -84,12 +80,11 @@ CometNativeColumnarToRow : : : : : +- CometBroadcastHashJoin : : : : : :- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometNativeColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : +- CometSubqueryBroadcast + : : : : : : +- CometBroadcastExchange + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : : +- CometBroadcastExchange : : : : : +- CometBroadcastHashJoin : : : : : :- CometFilter @@ -101,12 +96,11 @@ CometNativeColumnarToRow : : : : : : +- CometBroadcastHashJoin : : : : : : :- CometFilter : : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : : : : +- SubqueryBroadcast - : : : : : : : +- BroadcastExchange - : : : : : : : +- CometNativeColumnarToRow - : : : : : : : +- CometProject - : : : : : : : +- CometFilter - : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : : +- CometSubqueryBroadcast + : : : : : : : +- CometBroadcastExchange + : : : : : : : +- CometProject + : : : : : : : +- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : : : +- CometBroadcastExchange : : : : : : +- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.item @@ -125,12 +119,11 @@ CometNativeColumnarToRow : : : : +- CometBroadcastHashJoin : : : : :- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- CometSubqueryBroadcast + : : : : : +- CometBroadcastExchange + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : +- CometBroadcastExchange : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.item @@ -158,12 +151,11 @@ CometNativeColumnarToRow : : : : +- CometBroadcastHashJoin : : : : :- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- CometSubqueryBroadcast + : : : : : +- CometBroadcastExchange + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : +- CometBroadcastExchange : : : : +- CometBroadcastHashJoin : : : : :- CometFilter @@ -175,12 +167,11 @@ CometNativeColumnarToRow : : : : : +- CometBroadcastHashJoin : : : : : :- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometNativeColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : +- CometSubqueryBroadcast + : : : : : : +- CometBroadcastExchange + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : : +- CometBroadcastExchange : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.item @@ -199,12 +190,11 @@ CometNativeColumnarToRow : : : +- CometBroadcastHashJoin : : : :- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometSubqueryBroadcast + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- CometBroadcastExchange : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.item @@ -229,12 +219,11 @@ CometNativeColumnarToRow : : :- CometBroadcastHashJoin : : : :- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometSubqueryBroadcast + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometBroadcastHashJoin @@ -251,12 +240,11 @@ CometNativeColumnarToRow : : : : : +- CometBroadcastHashJoin : : : : : :- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometNativeColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : +- CometSubqueryBroadcast + : : : : : : +- CometBroadcastExchange + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : : +- CometBroadcastExchange : : : : : +- CometBroadcastHashJoin : : : : : :- CometFilter @@ -268,12 +256,11 @@ CometNativeColumnarToRow : : : : : : +- CometBroadcastHashJoin : : : : : : :- CometFilter : : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : : : : +- SubqueryBroadcast - : : : : : : : +- BroadcastExchange - : : : : : : : +- CometNativeColumnarToRow - : : : : : : : +- CometProject - : : : : : : : +- CometFilter - : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : : +- CometSubqueryBroadcast + : : : : : : : +- CometBroadcastExchange + : : : : : : : +- CometProject + : : : : : : : +- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : : : +- CometBroadcastExchange : : : : : : +- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.item @@ -292,12 +279,11 @@ CometNativeColumnarToRow : : : : +- CometBroadcastHashJoin : : : : :- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- CometSubqueryBroadcast + : : : : : +- CometBroadcastExchange + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : +- CometBroadcastExchange : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.item @@ -325,12 +311,11 @@ CometNativeColumnarToRow : : : : +- CometBroadcastHashJoin : : : : :- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- CometSubqueryBroadcast + : : : : : +- CometBroadcastExchange + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : +- CometBroadcastExchange : : : : +- CometBroadcastHashJoin : : : : :- CometFilter @@ -342,12 +327,11 @@ CometNativeColumnarToRow : : : : : +- CometBroadcastHashJoin : : : : : :- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometNativeColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : +- CometSubqueryBroadcast + : : : : : : +- CometBroadcastExchange + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : : +- CometBroadcastExchange : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.item @@ -366,12 +350,11 @@ CometNativeColumnarToRow : : : +- CometBroadcastHashJoin : : : :- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometSubqueryBroadcast + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- CometBroadcastExchange : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.item @@ -396,12 +379,11 @@ CometNativeColumnarToRow : :- CometBroadcastHashJoin : : :- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometSubqueryBroadcast + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- CometBroadcastExchange : : +- CometProject : : +- CometBroadcastHashJoin @@ -418,12 +400,11 @@ CometNativeColumnarToRow : : : : +- CometBroadcastHashJoin : : : : :- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- CometSubqueryBroadcast + : : : : : +- CometBroadcastExchange + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : +- CometBroadcastExchange : : : : +- CometBroadcastHashJoin : : : : :- CometFilter @@ -435,12 +416,11 @@ CometNativeColumnarToRow : : : : : +- CometBroadcastHashJoin : : : : : :- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometNativeColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : +- CometSubqueryBroadcast + : : : : : : +- CometBroadcastExchange + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : : +- CometBroadcastExchange : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.item @@ -459,12 +439,11 @@ CometNativeColumnarToRow : : : +- CometBroadcastHashJoin : : : :- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometSubqueryBroadcast + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- CometBroadcastExchange : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.item @@ -492,12 +471,11 @@ CometNativeColumnarToRow : : : +- CometBroadcastHashJoin : : : :- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometSubqueryBroadcast + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- CometBroadcastExchange : : : +- CometBroadcastHashJoin : : : :- CometFilter @@ -509,12 +487,11 @@ CometNativeColumnarToRow : : : : +- CometBroadcastHashJoin : : : : :- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- CometSubqueryBroadcast + : : : : : +- CometBroadcastExchange + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : +- CometBroadcastExchange : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.item @@ -533,12 +510,11 @@ CometNativeColumnarToRow : : +- CometBroadcastHashJoin : : :- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometSubqueryBroadcast + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- CometBroadcastExchange : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.item @@ -551,4 +527,4 @@ CometNativeColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.date_dim -Comet accelerated 475 out of 526 eligible operators (90%). Final plan contains 26 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 499 out of 526 eligible operators (94%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q14b.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q14b.native_datafusion/extended.txt index b48995c0a5..341ae1fb04 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q14b.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q14b.native_datafusion/extended.txt @@ -11,12 +11,11 @@ CometNativeColumnarToRow : : :- CometProject : : : +- CometBroadcastHashJoin : : : :- CometNativeScan parquet spark_catalog.default.store_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometSubqueryBroadcast + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter @@ -24,12 +23,11 @@ CometNativeColumnarToRow : : :- CometProject : : : +- CometBroadcastHashJoin : : : :- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometSubqueryBroadcast + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter @@ -37,12 +35,11 @@ CometNativeColumnarToRow : : +- CometProject : : +- CometBroadcastHashJoin : : :- CometNativeScan parquet spark_catalog.default.web_sales - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometSubqueryBroadcast + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- CometBroadcastExchange : : +- CometProject : : +- CometFilter @@ -57,22 +54,21 @@ CometNativeColumnarToRow : : :- CometBroadcastHashJoin : : : :- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : : +- Subquery - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : +- Subquery - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometSubqueryBroadcast + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : : +- Subquery + : : : : : +- CometNativeColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- Subquery + : : : : +- CometNativeColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometBroadcastHashJoin @@ -89,12 +85,11 @@ CometNativeColumnarToRow : : : : : +- CometBroadcastHashJoin : : : : : :- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometNativeColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : +- CometSubqueryBroadcast + : : : : : : +- CometBroadcastExchange + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : : +- CometBroadcastExchange : : : : : +- CometBroadcastHashJoin : : : : : :- CometFilter @@ -106,12 +101,11 @@ CometNativeColumnarToRow : : : : : : +- CometBroadcastHashJoin : : : : : : :- CometFilter : : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : : : : +- SubqueryBroadcast - : : : : : : : +- BroadcastExchange - : : : : : : : +- CometNativeColumnarToRow - : : : : : : : +- CometProject - : : : : : : : +- CometFilter - : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : : +- CometSubqueryBroadcast + : : : : : : : +- CometBroadcastExchange + : : : : : : : +- CometProject + : : : : : : : +- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : : : +- CometBroadcastExchange : : : : : : +- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.item @@ -130,12 +124,11 @@ CometNativeColumnarToRow : : : : +- CometBroadcastHashJoin : : : : :- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- CometSubqueryBroadcast + : : : : : +- CometBroadcastExchange + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : +- CometBroadcastExchange : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.item @@ -163,12 +156,11 @@ CometNativeColumnarToRow : : : : +- CometBroadcastHashJoin : : : : :- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- CometSubqueryBroadcast + : : : : : +- CometBroadcastExchange + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : +- CometBroadcastExchange : : : : +- CometBroadcastHashJoin : : : : :- CometFilter @@ -180,12 +172,11 @@ CometNativeColumnarToRow : : : : : +- CometBroadcastHashJoin : : : : : :- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometNativeColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : +- CometSubqueryBroadcast + : : : : : : +- CometBroadcastExchange + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : : +- CometBroadcastExchange : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.item @@ -204,12 +195,11 @@ CometNativeColumnarToRow : : : +- CometBroadcastHashJoin : : : :- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometSubqueryBroadcast + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- CometBroadcastExchange : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.item @@ -240,22 +230,21 @@ CometNativeColumnarToRow : :- CometBroadcastHashJoin : : :- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : : +- Subquery - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- Subquery - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometSubqueryBroadcast + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : : +- Subquery + : : : : +- CometNativeColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- Subquery + : : : +- CometNativeColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- CometBroadcastExchange : : +- CometProject : : +- CometBroadcastHashJoin @@ -272,12 +261,11 @@ CometNativeColumnarToRow : : : : +- CometBroadcastHashJoin : : : : :- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- CometSubqueryBroadcast + : : : : : +- CometBroadcastExchange + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : +- CometBroadcastExchange : : : : +- CometBroadcastHashJoin : : : : :- CometFilter @@ -289,12 +277,11 @@ CometNativeColumnarToRow : : : : : +- CometBroadcastHashJoin : : : : : :- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometNativeColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : +- CometSubqueryBroadcast + : : : : : : +- CometBroadcastExchange + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : : +- CometBroadcastExchange : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.item @@ -313,12 +300,11 @@ CometNativeColumnarToRow : : : +- CometBroadcastHashJoin : : : :- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometSubqueryBroadcast + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- CometBroadcastExchange : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.item @@ -346,12 +332,11 @@ CometNativeColumnarToRow : : : +- CometBroadcastHashJoin : : : :- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometSubqueryBroadcast + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- CometBroadcastExchange : : : +- CometBroadcastHashJoin : : : :- CometFilter @@ -363,12 +348,11 @@ CometNativeColumnarToRow : : : : +- CometBroadcastHashJoin : : : : :- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- CometSubqueryBroadcast + : : : : : +- CometBroadcastExchange + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : +- CometBroadcastExchange : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.item @@ -387,12 +371,11 @@ CometNativeColumnarToRow : : +- CometBroadcastHashJoin : : :- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometSubqueryBroadcast + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- CometBroadcastExchange : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.item @@ -411,4 +394,4 @@ CometNativeColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.date_dim -Comet accelerated 343 out of 387 eligible operators (88%). Final plan contains 25 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 360 out of 387 eligible operators (93%). Final plan contains 8 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q15.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q15.native_datafusion/extended.txt index 42432bdc12..da1e91c969 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q15.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q15.native_datafusion/extended.txt @@ -11,12 +11,11 @@ CometNativeColumnarToRow : : +- CometBroadcastHashJoin : : :- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometSubqueryBroadcast + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- CometBroadcastExchange : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.customer @@ -29,4 +28,4 @@ CometNativeColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.date_dim -Comet accelerated 26 out of 28 eligible operators (92%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 27 out of 28 eligible operators (96%). Final plan contains 1 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q17.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q17.native_datafusion/extended.txt index f58c012650..2650cd8f1a 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q17.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q17.native_datafusion/extended.txt @@ -19,30 +19,27 @@ CometNativeColumnarToRow : : : : : : +- CometBroadcastHashJoin : : : : : : :- CometFilter : : : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : : : : : +- SubqueryBroadcast - : : : : : : : +- BroadcastExchange - : : : : : : : +- CometNativeColumnarToRow - : : : : : : : +- CometProject - : : : : : : : +- CometFilter - : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : : +- CometSubqueryBroadcast + : : : : : : : +- CometBroadcastExchange + : : : : : : : +- CometProject + : : : : : : : +- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : : : +- CometBroadcastExchange : : : : : : +- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.store_returns - : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometNativeColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : +- CometSubqueryBroadcast + : : : : : : +- CometBroadcastExchange + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : : +- CometBroadcastExchange : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- CometSubqueryBroadcast + : : : : : +- CometBroadcastExchange + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : +- CometBroadcastExchange : : : : +- CometProject : : : : +- CometFilter @@ -64,4 +61,4 @@ CometNativeColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.item -Comet accelerated 55 out of 61 eligible operators (90%). Final plan contains 4 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 58 out of 61 eligible operators (95%). Final plan contains 1 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q18.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q18.native_datafusion/extended.txt index b3485a3421..4040a49d5a 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q18.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q18.native_datafusion/extended.txt @@ -18,12 +18,11 @@ CometNativeColumnarToRow : : : : : +- CometBroadcastHashJoin : : : : : :- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometNativeColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : +- CometSubqueryBroadcast + : : : : : : +- CometBroadcastExchange + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : : +- CometBroadcastExchange : : : : : +- CometProject : : : : : +- CometFilter @@ -48,4 +47,4 @@ CometNativeColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.item -Comet accelerated 45 out of 47 eligible operators (95%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 46 out of 47 eligible operators (97%). Final plan contains 1 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q20.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q20.native_datafusion/extended.txt index c4f54f4289..24498162ca 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q20.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q20.native_datafusion/extended.txt @@ -13,12 +13,11 @@ TakeOrderedAndProject : +- CometBroadcastHashJoin : :- CometFilter : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometSubqueryBroadcast + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim : +- CometBroadcastExchange : +- CometProject : +- CometFilter @@ -28,4 +27,4 @@ TakeOrderedAndProject +- CometFilter +- CometNativeScan parquet spark_catalog.default.date_dim -Comet accelerated 22 out of 27 eligible operators (81%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 23 out of 27 eligible operators (85%). Final plan contains 1 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q21.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q21.native_datafusion/extended.txt index 8522eb4ab9..9cbd136255 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q21.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q21.native_datafusion/extended.txt @@ -12,11 +12,10 @@ CometNativeColumnarToRow : : +- CometBroadcastHashJoin : : :- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.inventory - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometSubqueryBroadcast + : : : +- CometBroadcastExchange + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- CometBroadcastExchange : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.warehouse @@ -28,4 +27,4 @@ CometNativeColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.date_dim -Comet accelerated 25 out of 27 eligible operators (92%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 26 out of 27 eligible operators (96%). Final plan contains 1 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q22.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q22.native_datafusion/extended.txt index 6b78d21629..36a0eb0c8a 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q22.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q22.native_datafusion/extended.txt @@ -12,12 +12,11 @@ CometNativeColumnarToRow : : +- CometBroadcastHashJoin : : :- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.inventory - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometSubqueryBroadcast + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- CometBroadcastExchange : : +- CometProject : : +- CometFilter @@ -30,4 +29,4 @@ CometNativeColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.warehouse -Comet accelerated 27 out of 29 eligible operators (93%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 28 out of 29 eligible operators (96%). Final plan contains 1 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q23a.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q23a.native_datafusion/extended.txt index 8a9990d536..3e46e4a329 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q23a.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q23a.native_datafusion/extended.txt @@ -12,12 +12,11 @@ CometNativeColumnarToRow : : : +- CometProject : : : +- CometBroadcastHashJoin : : : :- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometSubqueryBroadcast + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter @@ -30,12 +29,11 @@ CometNativeColumnarToRow : : : : +- CometBroadcastHashJoin : : : : :- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- CometSubqueryBroadcast + : : : : : +- CometBroadcastExchange + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : +- CometBroadcastExchange : : : : +- CometProject : : : : +- CometFilter @@ -60,12 +58,11 @@ CometNativeColumnarToRow : : : : +- CometBroadcastHashJoin : : : : :- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- CometSubqueryBroadcast + : : : : : +- CometBroadcastExchange + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : +- CometBroadcastExchange : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.customer @@ -97,12 +94,11 @@ CometNativeColumnarToRow : : +- CometProject : : +- CometBroadcastHashJoin : : :- CometNativeScan parquet spark_catalog.default.web_sales - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometSubqueryBroadcast + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- CometBroadcastExchange : : +- CometProject : : +- CometFilter @@ -115,12 +111,11 @@ CometNativeColumnarToRow : : : +- CometBroadcastHashJoin : : : :- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometSubqueryBroadcast + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter @@ -148,4 +143,4 @@ CometNativeColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.date_dim -Comet accelerated 130 out of 142 eligible operators (91%). Final plan contains 7 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 135 out of 142 eligible operators (95%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q23b.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q23b.native_datafusion/extended.txt index 52090c668e..79baaf1aa9 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q23b.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q23b.native_datafusion/extended.txt @@ -15,12 +15,11 @@ CometNativeColumnarToRow : : : : +- CometBroadcastHashJoin : : : : :- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- CometSubqueryBroadcast + : : : : : +- CometBroadcastExchange + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : +- CometBroadcastExchange : : : : +- CometProject : : : : +- CometFilter @@ -33,12 +32,11 @@ CometNativeColumnarToRow : : : : : +- CometBroadcastHashJoin : : : : : :- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometNativeColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : +- CometSubqueryBroadcast + : : : : : : +- CometBroadcastExchange + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : : +- CometBroadcastExchange : : : : : +- CometProject : : : : : +- CometFilter @@ -63,12 +61,11 @@ CometNativeColumnarToRow : : : : : +- CometBroadcastHashJoin : : : : : :- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometNativeColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : +- CometSubqueryBroadcast + : : : : : : +- CometBroadcastExchange + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : : +- CometBroadcastExchange : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.customer @@ -127,12 +124,11 @@ CometNativeColumnarToRow : : : +- CometBroadcastHashJoin : : : :- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometSubqueryBroadcast + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter @@ -145,12 +141,11 @@ CometNativeColumnarToRow : : : : +- CometBroadcastHashJoin : : : : :- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- CometSubqueryBroadcast + : : : : : +- CometBroadcastExchange + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : +- CometBroadcastExchange : : : : +- CometProject : : : : +- CometFilter @@ -200,4 +195,4 @@ CometNativeColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.date_dim -Comet accelerated 180 out of 194 eligible operators (92%). Final plan contains 7 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 185 out of 194 eligible operators (95%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q25.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q25.native_datafusion/extended.txt index f58c012650..2650cd8f1a 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q25.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q25.native_datafusion/extended.txt @@ -19,30 +19,27 @@ CometNativeColumnarToRow : : : : : : +- CometBroadcastHashJoin : : : : : : :- CometFilter : : : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : : : : : +- SubqueryBroadcast - : : : : : : : +- BroadcastExchange - : : : : : : : +- CometNativeColumnarToRow - : : : : : : : +- CometProject - : : : : : : : +- CometFilter - : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : : +- CometSubqueryBroadcast + : : : : : : : +- CometBroadcastExchange + : : : : : : : +- CometProject + : : : : : : : +- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : : : +- CometBroadcastExchange : : : : : : +- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.store_returns - : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometNativeColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : +- CometSubqueryBroadcast + : : : : : : +- CometBroadcastExchange + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : : +- CometBroadcastExchange : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- CometSubqueryBroadcast + : : : : : +- CometBroadcastExchange + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : +- CometBroadcastExchange : : : : +- CometProject : : : : +- CometFilter @@ -64,4 +61,4 @@ CometNativeColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.item -Comet accelerated 55 out of 61 eligible operators (90%). Final plan contains 4 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 58 out of 61 eligible operators (95%). Final plan contains 1 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q26.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q26.native_datafusion/extended.txt index 71fd3bbe6f..2f7fe7b668 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q26.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q26.native_datafusion/extended.txt @@ -13,12 +13,11 @@ CometNativeColumnarToRow : : : +- CometBroadcastHashJoin : : : :- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometSubqueryBroadcast + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter @@ -36,4 +35,4 @@ CometNativeColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.promotion -Comet accelerated 33 out of 35 eligible operators (94%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 34 out of 35 eligible operators (97%). Final plan contains 1 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q27.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q27.native_datafusion/extended.txt index 15ab28f784..df27dd568f 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q27.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q27.native_datafusion/extended.txt @@ -14,12 +14,11 @@ CometNativeColumnarToRow : : : +- CometBroadcastHashJoin : : : :- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometSubqueryBroadcast + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter @@ -37,4 +36,4 @@ CometNativeColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.item -Comet accelerated 34 out of 36 eligible operators (94%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 35 out of 36 eligible operators (97%). Final plan contains 1 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q29.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q29.native_datafusion/extended.txt index f58c012650..2650cd8f1a 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q29.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q29.native_datafusion/extended.txt @@ -19,30 +19,27 @@ CometNativeColumnarToRow : : : : : : +- CometBroadcastHashJoin : : : : : : :- CometFilter : : : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : : : : : +- SubqueryBroadcast - : : : : : : : +- BroadcastExchange - : : : : : : : +- CometNativeColumnarToRow - : : : : : : : +- CometProject - : : : : : : : +- CometFilter - : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : : +- CometSubqueryBroadcast + : : : : : : : +- CometBroadcastExchange + : : : : : : : +- CometProject + : : : : : : : +- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : : : +- CometBroadcastExchange : : : : : : +- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.store_returns - : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometNativeColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : +- CometSubqueryBroadcast + : : : : : : +- CometBroadcastExchange + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : : +- CometBroadcastExchange : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- CometSubqueryBroadcast + : : : : : +- CometBroadcastExchange + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : +- CometBroadcastExchange : : : : +- CometProject : : : : +- CometFilter @@ -64,4 +61,4 @@ CometNativeColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.item -Comet accelerated 55 out of 61 eligible operators (90%). Final plan contains 4 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 58 out of 61 eligible operators (95%). Final plan contains 1 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q30.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q30.native_datafusion/extended.txt index 18d0a12a03..2a22518df4 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q30.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q30.native_datafusion/extended.txt @@ -16,12 +16,11 @@ CometNativeColumnarToRow : : : : +- CometBroadcastHashJoin : : : : :- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.web_returns - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- CometSubqueryBroadcast + : : : : : +- CometBroadcastExchange + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : +- CometBroadcastExchange : : : : +- CometProject : : : : +- CometFilter @@ -44,12 +43,11 @@ CometNativeColumnarToRow : : : +- CometBroadcastHashJoin : : : :- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.web_returns - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometSubqueryBroadcast + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter @@ -67,4 +65,4 @@ CometNativeColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.customer_address -Comet accelerated 61 out of 65 eligible operators (93%). Final plan contains 3 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 63 out of 65 eligible operators (96%). Final plan contains 1 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q31.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q31.native_datafusion/extended.txt index d747a5243e..bf4fb2aa77 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q31.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q31.native_datafusion/extended.txt @@ -18,11 +18,10 @@ CometNativeColumnarToRow : : : : : : +- CometBroadcastHashJoin : : : : : : :- CometFilter : : : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : : : : : +- SubqueryBroadcast - : : : : : : : +- BroadcastExchange - : : : : : : : +- CometNativeColumnarToRow - : : : : : : : +- CometFilter - : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : : +- CometSubqueryBroadcast + : : : : : : : +- CometBroadcastExchange + : : : : : : : +- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : : : +- CometBroadcastExchange : : : : : : +- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -39,11 +38,10 @@ CometNativeColumnarToRow : : : : : +- CometBroadcastHashJoin : : : : : :- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometNativeColumnarToRow - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : +- CometSubqueryBroadcast + : : : : : : +- CometBroadcastExchange + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : : +- CometBroadcastExchange : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -60,11 +58,10 @@ CometNativeColumnarToRow : : : : +- CometBroadcastHashJoin : : : : :- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- CometSubqueryBroadcast + : : : : : +- CometBroadcastExchange + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : +- CometBroadcastExchange : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -81,11 +78,10 @@ CometNativeColumnarToRow : : : +- CometBroadcastHashJoin : : : :- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometSubqueryBroadcast + : : : : +- CometBroadcastExchange + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- CometBroadcastExchange : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -102,11 +98,10 @@ CometNativeColumnarToRow : : +- CometBroadcastHashJoin : : :- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometSubqueryBroadcast + : : : +- CometBroadcastExchange + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- CometBroadcastExchange : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -123,11 +118,10 @@ CometNativeColumnarToRow : +- CometBroadcastHashJoin : :- CometFilter : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometSubqueryBroadcast + : : +- CometBroadcastExchange + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim : +- CometBroadcastExchange : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -135,4 +129,4 @@ CometNativeColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.customer_address -Comet accelerated 117 out of 129 eligible operators (90%). Final plan contains 7 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 123 out of 129 eligible operators (95%). Final plan contains 1 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q32.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q32.native_datafusion/extended.txt index 5d009dea2d..db07e648bc 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q32.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q32.native_datafusion/extended.txt @@ -10,12 +10,11 @@ CometNativeColumnarToRow : : +- CometBroadcastHashJoin : : :- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometSubqueryBroadcast + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- CometBroadcastExchange : : +- CometProject : : +- CometFilter @@ -29,12 +28,11 @@ CometNativeColumnarToRow : +- CometBroadcastHashJoin : :- CometFilter : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometSubqueryBroadcast + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim : +- CometBroadcastExchange : +- CometProject : +- CometFilter @@ -44,4 +42,4 @@ CometNativeColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.date_dim -Comet accelerated 38 out of 42 eligible operators (90%). Final plan contains 3 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 40 out of 42 eligible operators (95%). Final plan contains 1 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q33.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q33.native_datafusion/extended.txt index 6afaa68137..d64ba66952 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q33.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q33.native_datafusion/extended.txt @@ -15,12 +15,11 @@ CometNativeColumnarToRow : : : +- CometBroadcastHashJoin : : : :- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometSubqueryBroadcast + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter @@ -48,12 +47,11 @@ CometNativeColumnarToRow : : : +- CometBroadcastHashJoin : : : :- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometSubqueryBroadcast + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter @@ -81,12 +79,11 @@ CometNativeColumnarToRow : : +- CometBroadcastHashJoin : : :- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometSubqueryBroadcast + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- CometBroadcastExchange : : +- CometProject : : +- CometFilter @@ -104,4 +101,4 @@ CometNativeColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.item -Comet accelerated 95 out of 101 eligible operators (94%). Final plan contains 4 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 98 out of 101 eligible operators (97%). Final plan contains 1 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q34.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q34.native_datafusion/extended.txt index 1b61be2937..4c0ef9abcd 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q34.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q34.native_datafusion/extended.txt @@ -15,12 +15,11 @@ CometNativeColumnarToRow : : : +- CometBroadcastHashJoin : : : :- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometSubqueryBroadcast + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter @@ -38,4 +37,4 @@ CometNativeColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.customer -Comet accelerated 35 out of 37 eligible operators (94%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 36 out of 37 eligible operators (97%). Final plan contains 1 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q35.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q35.native_datafusion/extended.txt index 2785c3d0b1..2e79f6f464 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q35.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q35.native_datafusion/extended.txt @@ -19,12 +19,11 @@ TakeOrderedAndProject : : : : +- CometProject : : : : +- CometBroadcastHashJoin : : : : :- CometNativeScan parquet spark_catalog.default.store_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- CometSubqueryBroadcast + : : : : : +- CometBroadcastExchange + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : +- CometBroadcastExchange : : : : +- CometProject : : : : +- CometFilter @@ -34,12 +33,11 @@ TakeOrderedAndProject : : : +- CometProject : : : +- CometBroadcastHashJoin : : : :- CometNativeScan parquet spark_catalog.default.web_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometSubqueryBroadcast + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter @@ -49,12 +47,11 @@ TakeOrderedAndProject : : +- CometProject : : +- CometBroadcastHashJoin : : :- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometSubqueryBroadcast + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- CometBroadcastExchange : : +- CometProject : : +- CometFilter @@ -70,4 +67,4 @@ TakeOrderedAndProject +- CometFilter +- CometNativeScan parquet spark_catalog.default.customer_demographics -Comet accelerated 41 out of 62 eligible operators (66%). Final plan contains 9 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 44 out of 62 eligible operators (70%). Final plan contains 6 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q36.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q36.native_datafusion/extended.txt index 05ce793d8d..9480047251 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q36.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q36.native_datafusion/extended.txt @@ -16,12 +16,11 @@ TakeOrderedAndProject : : +- CometBroadcastHashJoin : : :- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometSubqueryBroadcast + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- CometBroadcastExchange : : +- CometProject : : +- CometFilter @@ -35,4 +34,4 @@ TakeOrderedAndProject +- CometFilter +- CometNativeScan parquet spark_catalog.default.store -Comet accelerated 29 out of 34 eligible operators (85%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 30 out of 34 eligible operators (88%). Final plan contains 1 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q37.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q37.native_datafusion/extended.txt index 16e3a70d9b..a83c218727 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q37.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q37.native_datafusion/extended.txt @@ -17,12 +17,11 @@ CometNativeColumnarToRow : : +- CometProject : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.inventory - : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometSubqueryBroadcast + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim : +- CometBroadcastExchange : +- CometProject : +- CometFilter @@ -31,4 +30,4 @@ CometNativeColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.catalog_sales -Comet accelerated 28 out of 30 eligible operators (93%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 29 out of 30 eligible operators (96%). Final plan contains 1 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q38.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q38.native_datafusion/extended.txt index 29422273c5..c0314d8fb0 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q38.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q38.native_datafusion/extended.txt @@ -14,12 +14,11 @@ CometNativeColumnarToRow : : : +- CometBroadcastHashJoin : : : :- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometSubqueryBroadcast + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter @@ -38,12 +37,11 @@ CometNativeColumnarToRow : : +- CometBroadcastHashJoin : : :- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometSubqueryBroadcast + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- CometBroadcastExchange : : +- CometProject : : +- CometFilter @@ -62,12 +60,11 @@ CometNativeColumnarToRow : +- CometBroadcastHashJoin : :- CometFilter : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometSubqueryBroadcast + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim : +- CometBroadcastExchange : +- CometProject : +- CometFilter @@ -77,4 +74,4 @@ CometNativeColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.customer -Comet accelerated 68 out of 74 eligible operators (91%). Final plan contains 4 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 71 out of 74 eligible operators (95%). Final plan contains 1 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q39a.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q39a.native_datafusion/extended.txt index eba1ff4dcd..67ff950c02 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q39a.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q39a.native_datafusion/extended.txt @@ -15,12 +15,11 @@ CometNativeColumnarToRow : : : +- CometBroadcastHashJoin : : : :- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.inventory - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometSubqueryBroadcast + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- CometBroadcastExchange : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.item @@ -45,12 +44,11 @@ CometNativeColumnarToRow : : +- CometBroadcastHashJoin : : :- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.inventory - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometSubqueryBroadcast + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- CometBroadcastExchange : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.item @@ -62,4 +60,4 @@ CometNativeColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.date_dim -Comet accelerated 56 out of 60 eligible operators (93%). Final plan contains 3 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 58 out of 60 eligible operators (96%). Final plan contains 1 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q39b.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q39b.native_datafusion/extended.txt index eba1ff4dcd..67ff950c02 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q39b.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q39b.native_datafusion/extended.txt @@ -15,12 +15,11 @@ CometNativeColumnarToRow : : : +- CometBroadcastHashJoin : : : :- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.inventory - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometSubqueryBroadcast + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- CometBroadcastExchange : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.item @@ -45,12 +44,11 @@ CometNativeColumnarToRow : : +- CometBroadcastHashJoin : : :- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.inventory - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometSubqueryBroadcast + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- CometBroadcastExchange : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.item @@ -62,4 +60,4 @@ CometNativeColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.date_dim -Comet accelerated 56 out of 60 eligible operators (93%). Final plan contains 3 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 58 out of 60 eligible operators (96%). Final plan contains 1 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q4.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q4.native_datafusion/extended.txt index b2914584b4..bf61cfd81e 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q4.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q4.native_datafusion/extended.txt @@ -23,11 +23,10 @@ CometNativeColumnarToRow : : : : : : +- CometBroadcastExchange : : : : : : +- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometNativeColumnarToRow - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : +- CometSubqueryBroadcast + : : : : : : +- CometBroadcastExchange + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : : +- CometBroadcastExchange : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -45,11 +44,10 @@ CometNativeColumnarToRow : : : : : +- CometBroadcastExchange : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- CometSubqueryBroadcast + : : : : : +- CometBroadcastExchange + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : +- CometBroadcastExchange : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -68,11 +66,10 @@ CometNativeColumnarToRow : : : : +- CometBroadcastExchange : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometSubqueryBroadcast + : : : : +- CometBroadcastExchange + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- CometBroadcastExchange : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -90,11 +87,10 @@ CometNativeColumnarToRow : : : +- CometBroadcastExchange : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometSubqueryBroadcast + : : : +- CometBroadcastExchange + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- CometBroadcastExchange : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -113,11 +109,10 @@ CometNativeColumnarToRow : : +- CometBroadcastExchange : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometSubqueryBroadcast + : : +- CometBroadcastExchange + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim : +- CometBroadcastExchange : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -135,13 +130,12 @@ CometNativeColumnarToRow : +- CometBroadcastExchange : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.web_sales - : +- SubqueryBroadcast - : +- BroadcastExchange - : +- CometNativeColumnarToRow - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometSubqueryBroadcast + : +- CometBroadcastExchange + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.date_dim +- CometBroadcastExchange +- CometFilter +- CometNativeScan parquet spark_catalog.default.date_dim -Comet accelerated 126 out of 138 eligible operators (91%). Final plan contains 7 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 132 out of 138 eligible operators (95%). Final plan contains 1 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q40.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q40.native_datafusion/extended.txt index 9b00a2c6a4..1a148d7ab4 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q40.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q40.native_datafusion/extended.txt @@ -15,11 +15,10 @@ CometNativeColumnarToRow : : : : +- CometExchange : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometSubqueryBroadcast + : : : : +- CometBroadcastExchange + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- CometSort : : : +- CometExchange : : : +- CometProject @@ -37,4 +36,4 @@ CometNativeColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.date_dim -Comet accelerated 34 out of 36 eligible operators (94%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 35 out of 36 eligible operators (97%). Final plan contains 1 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q45.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q45.native_datafusion/extended.txt index 7292a238d5..e7658f2548 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q45.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q45.native_datafusion/extended.txt @@ -17,12 +17,11 @@ TakeOrderedAndProject : : : : +- CometBroadcastHashJoin : : : : :- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- CometSubqueryBroadcast + : : : : : +- CometBroadcastExchange + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : +- CometBroadcastExchange : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.customer @@ -44,4 +43,4 @@ TakeOrderedAndProject +- CometFilter +- CometNativeScan parquet spark_catalog.default.item -Comet accelerated 32 out of 41 eligible operators (78%). Final plan contains 4 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 33 out of 41 eligible operators (80%). Final plan contains 3 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q46.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q46.native_datafusion/extended.txt index dbdaaa275a..f58213e7cb 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q46.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q46.native_datafusion/extended.txt @@ -17,12 +17,11 @@ CometNativeColumnarToRow : : : : : +- CometBroadcastHashJoin : : : : : :- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometNativeColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : +- CometSubqueryBroadcast + : : : : : : +- CometBroadcastExchange + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : : +- CometBroadcastExchange : : : : : +- CometProject : : : : : +- CometFilter @@ -46,4 +45,4 @@ CometNativeColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.customer_address -Comet accelerated 43 out of 45 eligible operators (95%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 44 out of 45 eligible operators (97%). Final plan contains 1 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q47.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q47.native_datafusion/extended.txt index 0886059452..54fa1daf2e 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q47.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q47.native_datafusion/extended.txt @@ -26,11 +26,10 @@ TakeOrderedAndProject : : : : +- CometBroadcastExchange : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometSubqueryBroadcast + : : : : +- CometBroadcastExchange + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- CometBroadcastExchange : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -58,11 +57,10 @@ TakeOrderedAndProject : : : +- CometBroadcastExchange : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometSubqueryBroadcast + : : : +- CometBroadcastExchange + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- CometBroadcastExchange : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -90,11 +88,10 @@ TakeOrderedAndProject : : +- CometBroadcastExchange : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometSubqueryBroadcast + : : +- CometBroadcastExchange + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim : +- CometBroadcastExchange : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -102,4 +99,4 @@ TakeOrderedAndProject +- CometFilter +- CometNativeScan parquet spark_catalog.default.store -Comet accelerated 75 out of 97 eligible operators (77%). Final plan contains 6 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 78 out of 97 eligible operators (80%). Final plan contains 3 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q48.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q48.native_datafusion/extended.txt index 75a1c725c2..4b1541c674 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q48.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q48.native_datafusion/extended.txt @@ -12,12 +12,11 @@ CometNativeColumnarToRow : : : +- CometBroadcastHashJoin : : : :- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometSubqueryBroadcast + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- CometBroadcastExchange : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.store @@ -34,4 +33,4 @@ CometNativeColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.date_dim -Comet accelerated 31 out of 33 eligible operators (93%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 32 out of 33 eligible operators (96%). Final plan contains 1 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q49.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q49.native_datafusion/extended.txt index 6d7dadcde9..f2ae44fb86 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q49.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q49.native_datafusion/extended.txt @@ -23,12 +23,11 @@ CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometSubqueryBroadcast + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- CometProject : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.web_returns @@ -55,12 +54,11 @@ CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometSubqueryBroadcast + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- CometProject : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.catalog_returns @@ -87,12 +85,11 @@ CometNativeColumnarToRow : : +- CometProject : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometSubqueryBroadcast + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim : +- CometProject : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.store_returns @@ -101,4 +98,4 @@ CometNativeColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.date_dim -Comet accelerated 72 out of 95 eligible operators (75%). Final plan contains 7 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 75 out of 95 eligible operators (78%). Final plan contains 4 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q5.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q5.native_datafusion/extended.txt index 719229a9a3..adef074558 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q5.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q5.native_datafusion/extended.txt @@ -16,21 +16,19 @@ CometNativeColumnarToRow : : : :- CometProject : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometSubqueryBroadcast + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.store_returns - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometSubqueryBroadcast + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- CometBroadcastExchange : : +- CometProject : : +- CometFilter @@ -50,21 +48,19 @@ CometNativeColumnarToRow : : : :- CometProject : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometSubqueryBroadcast + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.catalog_returns - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometSubqueryBroadcast + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- CometBroadcastExchange : : +- CometProject : : +- CometFilter @@ -84,22 +80,20 @@ CometNativeColumnarToRow : : :- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometSubqueryBroadcast + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- CometProject : : +- CometBroadcastHashJoin : : :- CometBroadcastExchange : : : +- CometNativeScan parquet spark_catalog.default.web_returns - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometSubqueryBroadcast + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- CometProject : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.web_sales @@ -112,4 +106,4 @@ CometNativeColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.web_site -Comet accelerated 94 out of 106 eligible operators (88%). Final plan contains 7 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 100 out of 106 eligible operators (94%). Final plan contains 1 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q50.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q50.native_datafusion/extended.txt index c013381de0..e20ab4a6d6 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q50.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q50.native_datafusion/extended.txt @@ -16,12 +16,11 @@ CometNativeColumnarToRow : : : +- CometBroadcastExchange : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.store_returns - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometSubqueryBroadcast + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- CometBroadcastExchange : : +- CometProject : : +- CometFilter @@ -34,4 +33,4 @@ CometNativeColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.date_dim -Comet accelerated 31 out of 33 eligible operators (93%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 32 out of 33 eligible operators (96%). Final plan contains 1 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q51.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q51.native_datafusion/extended.txt index 425e772118..173d1f064f 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q51.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q51.native_datafusion/extended.txt @@ -20,12 +20,11 @@ TakeOrderedAndProject : +- CometBroadcastHashJoin : :- CometFilter : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometSubqueryBroadcast + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim : +- CometBroadcastExchange : +- CometProject : +- CometFilter @@ -44,15 +43,14 @@ TakeOrderedAndProject +- CometBroadcastHashJoin :- CometFilter : +- CometNativeScan parquet spark_catalog.default.store_sales - : +- SubqueryBroadcast - : +- BroadcastExchange - : +- CometNativeColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometSubqueryBroadcast + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.date_dim +- CometBroadcastExchange +- CometProject +- CometFilter +- CometNativeScan parquet spark_catalog.default.date_dim -Comet accelerated 40 out of 51 eligible operators (78%). Final plan contains 5 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 42 out of 51 eligible operators (82%). Final plan contains 3 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q53.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q53.native_datafusion/extended.txt index 936cde052f..9a8a9bbba4 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q53.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q53.native_datafusion/extended.txt @@ -20,12 +20,11 @@ TakeOrderedAndProject : : +- CometBroadcastExchange : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometSubqueryBroadcast + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim : +- CometBroadcastExchange : +- CometProject : +- CometFilter @@ -34,4 +33,4 @@ TakeOrderedAndProject +- CometFilter +- CometNativeScan parquet spark_catalog.default.store -Comet accelerated 27 out of 33 eligible operators (81%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 28 out of 33 eligible operators (84%). Final plan contains 1 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q54.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q54.native_datafusion/extended.txt index e402b1ddb0..cfb68d7bfb 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q54.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q54.native_datafusion/extended.txt @@ -27,21 +27,19 @@ CometNativeColumnarToRow : : : : : : : :- CometProject : : : : : : : : +- CometFilter : : : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : : : : : +- SubqueryBroadcast - : : : : : : : : +- BroadcastExchange - : : : : : : : : +- CometNativeColumnarToRow - : : : : : : : : +- CometProject - : : : : : : : : +- CometFilter - : : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : : : +- CometSubqueryBroadcast + : : : : : : : : +- CometBroadcastExchange + : : : : : : : : +- CometProject + : : : : : : : : +- CometFilter + : : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : : : : +- CometProject : : : : : : : +- CometFilter : : : : : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : : : : : +- SubqueryBroadcast - : : : : : : : +- BroadcastExchange - : : : : : : : +- CometNativeColumnarToRow - : : : : : : : +- CometProject - : : : : : : : +- CometFilter - : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : : +- CometSubqueryBroadcast + : : : : : : : +- CometBroadcastExchange + : : : : : : : +- CometProject + : : : : : : : +- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : : : +- CometBroadcastExchange : : : : : : +- CometProject : : : : : : +- CometFilter @@ -56,44 +54,43 @@ CometNativeColumnarToRow : : : +- CometBroadcastExchange : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : : :- Subquery - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometHashAggregate - : : : : : +- CometExchange - : : : : : +- CometHashAggregate - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : +- Subquery - : : : : +- CometNativeColumnarToRow - : : : : +- CometHashAggregate - : : : : +- CometExchange - : : : : +- CometHashAggregate - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : :- Subquery - : : : : +- CometNativeColumnarToRow - : : : : +- CometHashAggregate - : : : : +- CometExchange - : : : : +- CometHashAggregate - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- Subquery - : : : +- CometNativeColumnarToRow - : : : +- CometHashAggregate - : : : +- CometExchange - : : : +- CometHashAggregate - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometSubqueryBroadcast + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : : :- Subquery + : : : : : +- CometNativeColumnarToRow + : : : : : +- CometHashAggregate + : : : : : +- CometExchange + : : : : : +- CometHashAggregate + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- Subquery + : : : : +- CometNativeColumnarToRow + : : : : +- CometHashAggregate + : : : : +- CometExchange + : : : : +- CometHashAggregate + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : :- Subquery + : : : : +- CometNativeColumnarToRow + : : : : +- CometHashAggregate + : : : : +- CometExchange + : : : : +- CometHashAggregate + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- Subquery + : : : +- CometNativeColumnarToRow + : : : +- CometHashAggregate + : : : +- CometExchange + : : : +- CometHashAggregate + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- CometBroadcastExchange : : +- CometProject : : +- CometFilter @@ -125,4 +122,4 @@ CometNativeColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.date_dim -Comet accelerated 102 out of 116 eligible operators (87%). Final plan contains 10 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 105 out of 116 eligible operators (90%). Final plan contains 7 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q56.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q56.native_datafusion/extended.txt index d52c40d57b..0744bc307d 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q56.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q56.native_datafusion/extended.txt @@ -15,12 +15,11 @@ CometNativeColumnarToRow : : : +- CometBroadcastHashJoin : : : :- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometSubqueryBroadcast + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter @@ -49,12 +48,11 @@ CometNativeColumnarToRow : : : +- CometBroadcastHashJoin : : : :- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometSubqueryBroadcast + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter @@ -83,12 +81,11 @@ CometNativeColumnarToRow : : +- CometBroadcastHashJoin : : :- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometSubqueryBroadcast + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- CometBroadcastExchange : : +- CometProject : : +- CometFilter @@ -107,4 +104,4 @@ CometNativeColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.item -Comet accelerated 98 out of 104 eligible operators (94%). Final plan contains 4 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 101 out of 104 eligible operators (97%). Final plan contains 1 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q57.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q57.native_datafusion/extended.txt index 024bfeeeea..aa4ccf3246 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q57.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q57.native_datafusion/extended.txt @@ -26,11 +26,10 @@ TakeOrderedAndProject : : : : +- CometBroadcastExchange : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometSubqueryBroadcast + : : : : +- CometBroadcastExchange + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- CometBroadcastExchange : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -58,11 +57,10 @@ TakeOrderedAndProject : : : +- CometBroadcastExchange : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometSubqueryBroadcast + : : : +- CometBroadcastExchange + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- CometBroadcastExchange : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -90,11 +88,10 @@ TakeOrderedAndProject : : +- CometBroadcastExchange : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometSubqueryBroadcast + : : +- CometBroadcastExchange + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim : +- CometBroadcastExchange : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -102,4 +99,4 @@ TakeOrderedAndProject +- CometFilter +- CometNativeScan parquet spark_catalog.default.call_center -Comet accelerated 75 out of 97 eligible operators (77%). Final plan contains 6 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 78 out of 97 eligible operators (80%). Final plan contains 3 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q58.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q58.native_datafusion/extended.txt index b1e517e7d2..4969cdcdfa 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q58.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q58.native_datafusion/extended.txt @@ -14,27 +14,26 @@ CometNativeColumnarToRow : : : +- CometBroadcastHashJoin : : : :- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : : +- Subquery - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : +- Subquery - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometSubqueryBroadcast + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : : +- Subquery + : : : : : +- CometNativeColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- Subquery + : : : : +- CometNativeColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter @@ -65,27 +64,26 @@ CometNativeColumnarToRow : : +- CometBroadcastHashJoin : : :- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : : +- Subquery - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- Subquery - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometSubqueryBroadcast + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : : +- Subquery + : : : : +- CometNativeColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- Subquery + : : : +- CometNativeColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- CometBroadcastExchange : : +- CometProject : : +- CometFilter @@ -116,27 +114,26 @@ CometNativeColumnarToRow : +- CometBroadcastHashJoin : :- CometFilter : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : : +- Subquery - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- Subquery - : : +- CometNativeColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometSubqueryBroadcast + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : : +- Subquery + : : : +- CometNativeColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- Subquery + : : +- CometNativeColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim : +- CometBroadcastExchange : +- CometProject : +- CometFilter @@ -157,4 +154,4 @@ CometNativeColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.date_dim -Comet accelerated 127 out of 145 eligible operators (87%). Final plan contains 13 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 130 out of 145 eligible operators (89%). Final plan contains 10 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q6.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q6.native_datafusion/extended.txt index d5de400618..e8759a6e10 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q6.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q6.native_datafusion/extended.txt @@ -21,28 +21,27 @@ CometNativeColumnarToRow : : +- CometBroadcastExchange : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometProject - : : +- CometFilter - : : : +- Subquery - : : : +- CometNativeColumnarToRow - : : : +- CometHashAggregate - : : : +- CometExchange - : : : +- CometHashAggregate - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- Subquery - : : +- CometNativeColumnarToRow - : : +- CometHashAggregate - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometSubqueryBroadcast + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : : +- Subquery + : : : +- CometNativeColumnarToRow + : : : +- CometHashAggregate + : : : +- CometExchange + : : : +- CometHashAggregate + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- Subquery + : : +- CometNativeColumnarToRow + : : +- CometHashAggregate + : : +- CometExchange + : : +- CometHashAggregate + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim : +- CometBroadcastExchange : +- CometProject : +- CometFilter @@ -70,4 +69,4 @@ CometNativeColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.item -Comet accelerated 60 out of 66 eligible operators (90%). Final plan contains 5 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 61 out of 66 eligible operators (92%). Final plan contains 4 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q60.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q60.native_datafusion/extended.txt index d52c40d57b..0744bc307d 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q60.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q60.native_datafusion/extended.txt @@ -15,12 +15,11 @@ CometNativeColumnarToRow : : : +- CometBroadcastHashJoin : : : :- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometSubqueryBroadcast + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter @@ -49,12 +48,11 @@ CometNativeColumnarToRow : : : +- CometBroadcastHashJoin : : : :- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometSubqueryBroadcast + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter @@ -83,12 +81,11 @@ CometNativeColumnarToRow : : +- CometBroadcastHashJoin : : :- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometSubqueryBroadcast + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- CometBroadcastExchange : : +- CometProject : : +- CometFilter @@ -107,4 +104,4 @@ CometNativeColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.item -Comet accelerated 98 out of 104 eligible operators (94%). Final plan contains 4 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 101 out of 104 eligible operators (97%). Final plan contains 1 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q61.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q61.native_datafusion/extended.txt index ef8907c7d8..8d91cbe7a2 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q61.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q61.native_datafusion/extended.txt @@ -18,12 +18,11 @@ Project : : : : : : +- CometBroadcastHashJoin : : : : : : :- CometFilter : : : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : : : : : +- SubqueryBroadcast - : : : : : : : +- BroadcastExchange - : : : : : : : +- CometNativeColumnarToRow - : : : : : : : +- CometProject - : : : : : : : +- CometFilter - : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : : +- CometSubqueryBroadcast + : : : : : : : +- CometBroadcastExchange + : : : : : : : +- CometProject + : : : : : : : +- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : : : +- CometBroadcastExchange : : : : : : +- CometProject : : : : : : +- CometFilter @@ -64,12 +63,11 @@ Project : : : : +- CometBroadcastHashJoin : : : : :- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- CometSubqueryBroadcast + : : : : : +- CometBroadcastExchange + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : +- CometBroadcastExchange : : : : +- CometProject : : : : +- CometFilter @@ -90,4 +88,4 @@ Project +- CometFilter +- CometNativeScan parquet spark_catalog.default.item -Comet accelerated 80 out of 87 eligible operators (91%). Final plan contains 4 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 82 out of 87 eligible operators (94%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q63.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q63.native_datafusion/extended.txt index 936cde052f..9a8a9bbba4 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q63.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q63.native_datafusion/extended.txt @@ -20,12 +20,11 @@ TakeOrderedAndProject : : +- CometBroadcastExchange : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometSubqueryBroadcast + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim : +- CometBroadcastExchange : +- CometProject : +- CometFilter @@ -34,4 +33,4 @@ TakeOrderedAndProject +- CometFilter +- CometNativeScan parquet spark_catalog.default.store -Comet accelerated 27 out of 33 eligible operators (81%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 28 out of 33 eligible operators (84%). Final plan contains 1 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q64.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q64.native_datafusion/extended.txt index f1fe6fb3e3..14ffe4eab9 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q64.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q64.native_datafusion/extended.txt @@ -46,11 +46,10 @@ CometNativeColumnarToRow : : : : : : : : : : : : : : : : : :- CometBroadcastExchange : : : : : : : : : : : : : : : : : : +- CometFilter : : : : : : : : : : : : : : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : : : : : : : : : : : : : : : : +- SubqueryBroadcast - : : : : : : : : : : : : : : : : : : +- BroadcastExchange - : : : : : : : : : : : : : : : : : : +- CometNativeColumnarToRow - : : : : : : : : : : : : : : : : : : +- CometFilter - : : : : : : : : : : : : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : : : : : : : : : : : : : +- CometSubqueryBroadcast + : : : : : : : : : : : : : : : : : : +- CometBroadcastExchange + : : : : : : : : : : : : : : : : : : +- CometFilter + : : : : : : : : : : : : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : : : : : : : : : : : : : : +- CometProject : : : : : : : : : : : : : : : : : +- CometFilter : : : : : : : : : : : : : : : : : +- CometNativeScan parquet spark_catalog.default.store_returns @@ -166,11 +165,10 @@ CometNativeColumnarToRow : : : : : : : : : : : : : : : : :- CometBroadcastExchange : : : : : : : : : : : : : : : : : +- CometFilter : : : : : : : : : : : : : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : : : : : : : : : : : : : : : +- SubqueryBroadcast - : : : : : : : : : : : : : : : : : +- BroadcastExchange - : : : : : : : : : : : : : : : : : +- CometNativeColumnarToRow - : : : : : : : : : : : : : : : : : +- CometFilter - : : : : : : : : : : : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : : : : : : : : : : : : +- CometSubqueryBroadcast + : : : : : : : : : : : : : : : : : +- CometBroadcastExchange + : : : : : : : : : : : : : : : : : +- CometFilter + : : : : : : : : : : : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : : : : : : : : : : : : : +- CometProject : : : : : : : : : : : : : : : : +- CometFilter : : : : : : : : : : : : : : : : +- CometNativeScan parquet spark_catalog.default.store_returns @@ -244,4 +242,4 @@ CometNativeColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.item -Comet accelerated 238 out of 242 eligible operators (98%). Final plan contains 3 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 240 out of 242 eligible operators (99%). Final plan contains 1 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q65.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q65.native_datafusion/extended.txt index 167a3386c2..21d60497a1 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q65.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q65.native_datafusion/extended.txt @@ -17,12 +17,11 @@ CometNativeColumnarToRow : : +- CometBroadcastHashJoin : : :- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometSubqueryBroadcast + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- CometBroadcastExchange : : +- CometProject : : +- CometFilter @@ -43,15 +42,14 @@ CometNativeColumnarToRow +- CometBroadcastHashJoin :- CometFilter : +- CometNativeScan parquet spark_catalog.default.store_sales - : +- SubqueryBroadcast - : +- BroadcastExchange - : +- CometNativeColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometSubqueryBroadcast + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.date_dim +- CometBroadcastExchange +- CometProject +- CometFilter +- CometNativeScan parquet spark_catalog.default.date_dim -Comet accelerated 48 out of 52 eligible operators (92%). Final plan contains 3 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 50 out of 52 eligible operators (96%). Final plan contains 1 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q66.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q66.native_datafusion/extended.txt index a7423e3a59..c07ecfe46a 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q66.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q66.native_datafusion/extended.txt @@ -17,11 +17,10 @@ CometNativeColumnarToRow : : : : +- CometBroadcastHashJoin : : : : :- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- CometSubqueryBroadcast + : : : : : +- CometBroadcastExchange + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : +- CometBroadcastExchange : : : : +- CometProject : : : : +- CometFilter @@ -50,11 +49,10 @@ CometNativeColumnarToRow : : : +- CometBroadcastHashJoin : : : :- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometSubqueryBroadcast + : : : : +- CometBroadcastExchange + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter @@ -71,4 +69,4 @@ CometNativeColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.ship_mode -Comet accelerated 65 out of 69 eligible operators (94%). Final plan contains 3 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 67 out of 69 eligible operators (97%). Final plan contains 1 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q67.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q67.native_datafusion/extended.txt index 94c6cc711d..3c1bc08e97 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q67.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q67.native_datafusion/extended.txt @@ -20,12 +20,11 @@ TakeOrderedAndProject : : +- CometBroadcastHashJoin : : :- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometSubqueryBroadcast + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- CometBroadcastExchange : : +- CometProject : : +- CometFilter @@ -39,4 +38,4 @@ TakeOrderedAndProject +- CometFilter +- CometNativeScan parquet spark_catalog.default.item -Comet accelerated 30 out of 37 eligible operators (81%). Final plan contains 3 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 31 out of 37 eligible operators (83%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q68.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q68.native_datafusion/extended.txt index dbdaaa275a..f58213e7cb 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q68.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q68.native_datafusion/extended.txt @@ -17,12 +17,11 @@ CometNativeColumnarToRow : : : : : +- CometBroadcastHashJoin : : : : : :- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometNativeColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : +- CometSubqueryBroadcast + : : : : : : +- CometBroadcastExchange + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : : +- CometBroadcastExchange : : : : : +- CometProject : : : : : +- CometFilter @@ -46,4 +45,4 @@ CometNativeColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.customer_address -Comet accelerated 43 out of 45 eligible operators (95%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 44 out of 45 eligible operators (97%). Final plan contains 1 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q69.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q69.native_datafusion/extended.txt index 3e0ba48add..6d398580fb 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q69.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q69.native_datafusion/extended.txt @@ -18,12 +18,11 @@ TakeOrderedAndProject : : : : +- CometProject : : : : +- CometBroadcastHashJoin : : : : :- CometNativeScan parquet spark_catalog.default.store_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- CometSubqueryBroadcast + : : : : : +- CometBroadcastExchange + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : +- CometBroadcastExchange : : : : +- CometProject : : : : +- CometFilter @@ -33,12 +32,11 @@ TakeOrderedAndProject : : : +- CometProject : : : +- CometBroadcastHashJoin : : : :- CometNativeScan parquet spark_catalog.default.web_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometSubqueryBroadcast + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter @@ -48,12 +46,11 @@ TakeOrderedAndProject : : +- CometProject : : +- CometBroadcastHashJoin : : :- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometSubqueryBroadcast + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- CometBroadcastExchange : : +- CometProject : : +- CometFilter @@ -69,4 +66,4 @@ TakeOrderedAndProject +- CometFilter +- CometNativeScan parquet spark_catalog.default.customer_demographics -Comet accelerated 41 out of 61 eligible operators (67%). Final plan contains 9 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 44 out of 61 eligible operators (72%). Final plan contains 6 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q7.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q7.native_datafusion/extended.txt index 42206d1be5..8750081c55 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q7.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q7.native_datafusion/extended.txt @@ -13,12 +13,11 @@ CometNativeColumnarToRow : : : +- CometBroadcastHashJoin : : : :- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometSubqueryBroadcast + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter @@ -36,4 +35,4 @@ CometNativeColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.promotion -Comet accelerated 33 out of 35 eligible operators (94%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 34 out of 35 eligible operators (97%). Final plan contains 1 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q70.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q70.native_datafusion/extended.txt index befbf52ae5..d1ff1b903e 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q70.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q70.native_datafusion/extended.txt @@ -16,12 +16,11 @@ TakeOrderedAndProject : +- CometBroadcastHashJoin : :- CometFilter : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometSubqueryBroadcast + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim : +- CometBroadcastExchange : +- CometProject : +- CometFilter @@ -48,12 +47,11 @@ TakeOrderedAndProject : +- CometBroadcastHashJoin : :- CometFilter : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometSubqueryBroadcast + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim : +- CometBroadcastExchange : +- CometProject : +- CometFilter @@ -63,4 +61,4 @@ TakeOrderedAndProject +- CometFilter +- CometNativeScan parquet spark_catalog.default.date_dim -Comet accelerated 37 out of 57 eligible operators (64%). Final plan contains 7 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 39 out of 57 eligible operators (68%). Final plan contains 5 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q71.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q71.native_datafusion/extended.txt index 7f76dd192d..834098d4ee 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q71.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q71.native_datafusion/extended.txt @@ -17,12 +17,11 @@ CometNativeColumnarToRow : : +- CometBroadcastHashJoin : : :- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometSubqueryBroadcast + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- CometBroadcastExchange : : +- CometProject : : +- CometFilter @@ -31,12 +30,11 @@ CometNativeColumnarToRow : : +- CometBroadcastHashJoin : : :- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometSubqueryBroadcast + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- CometBroadcastExchange : : +- CometProject : : +- CometFilter @@ -45,12 +43,11 @@ CometNativeColumnarToRow : +- CometBroadcastHashJoin : :- CometFilter : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometSubqueryBroadcast + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim : +- CometBroadcastExchange : +- CometProject : +- CometFilter @@ -60,4 +57,4 @@ CometNativeColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.time_dim -Comet accelerated 51 out of 57 eligible operators (89%). Final plan contains 4 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 54 out of 57 eligible operators (94%). Final plan contains 1 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q72.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q72.native_datafusion/extended.txt index 623a68ddab..d443b31d1e 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q72.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q72.native_datafusion/extended.txt @@ -27,12 +27,11 @@ CometNativeColumnarToRow : : : : : : : : : +- CometBroadcastHashJoin : : : : : : : : : :- CometFilter : : : : : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : : : : : : : +- SubqueryBroadcast - : : : : : : : : : : +- BroadcastExchange - : : : : : : : : : : +- CometNativeColumnarToRow - : : : : : : : : : : +- CometProject - : : : : : : : : : : +- CometFilter - : : : : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : : : : : +- CometSubqueryBroadcast + : : : : : : : : : : +- CometBroadcastExchange + : : : : : : : : : : +- CometProject + : : : : : : : : : : +- CometFilter + : : : : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : : : : : : +- CometBroadcastExchange : : : : : : : : : +- CometFilter : : : : : : : : : +- CometNativeScan parquet spark_catalog.default.inventory @@ -69,4 +68,4 @@ CometNativeColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.catalog_returns -Comet accelerated 66 out of 68 eligible operators (97%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 67 out of 68 eligible operators (98%). Final plan contains 1 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q73.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q73.native_datafusion/extended.txt index 1b61be2937..4c0ef9abcd 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q73.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q73.native_datafusion/extended.txt @@ -15,12 +15,11 @@ CometNativeColumnarToRow : : : +- CometBroadcastHashJoin : : : :- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometSubqueryBroadcast + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter @@ -38,4 +37,4 @@ CometNativeColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.customer -Comet accelerated 35 out of 37 eligible operators (94%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 36 out of 37 eligible operators (97%). Final plan contains 1 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q74.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q74.native_datafusion/extended.txt index 51a3f5ca94..7768138645 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q74.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q74.native_datafusion/extended.txt @@ -19,11 +19,10 @@ CometNativeColumnarToRow : : : : +- CometBroadcastExchange : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometSubqueryBroadcast + : : : : +- CometBroadcastExchange + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- CometBroadcastExchange : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -41,11 +40,10 @@ CometNativeColumnarToRow : : : +- CometBroadcastExchange : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometSubqueryBroadcast + : : : +- CometBroadcastExchange + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- CometBroadcastExchange : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -64,11 +62,10 @@ CometNativeColumnarToRow : : +- CometBroadcastExchange : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometSubqueryBroadcast + : : +- CometBroadcastExchange + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim : +- CometBroadcastExchange : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -86,13 +83,12 @@ CometNativeColumnarToRow : +- CometBroadcastExchange : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.web_sales - : +- SubqueryBroadcast - : +- BroadcastExchange - : +- CometNativeColumnarToRow - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometSubqueryBroadcast + : +- CometBroadcastExchange + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.date_dim +- CometBroadcastExchange +- CometFilter +- CometNativeScan parquet spark_catalog.default.date_dim -Comet accelerated 83 out of 91 eligible operators (91%). Final plan contains 5 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 87 out of 91 eligible operators (95%). Final plan contains 1 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q75.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q75.native_datafusion/extended.txt index a943308f1e..ee7c0af216 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q75.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q75.native_datafusion/extended.txt @@ -22,11 +22,10 @@ CometNativeColumnarToRow : : : : +- CometBroadcastHashJoin : : : : :- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- CometSubqueryBroadcast + : : : : : +- CometBroadcastExchange + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : +- CometBroadcastExchange : : : : +- CometProject : : : : +- CometFilter @@ -49,11 +48,10 @@ CometNativeColumnarToRow : : : : +- CometBroadcastHashJoin : : : : :- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- CometSubqueryBroadcast + : : : : : +- CometBroadcastExchange + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : +- CometBroadcastExchange : : : : +- CometProject : : : : +- CometFilter @@ -76,11 +74,10 @@ CometNativeColumnarToRow : : : +- CometBroadcastHashJoin : : : :- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometSubqueryBroadcast + : : : : +- CometBroadcastExchange + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter @@ -113,11 +110,10 @@ CometNativeColumnarToRow : : : +- CometBroadcastHashJoin : : : :- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometSubqueryBroadcast + : : : : +- CometBroadcastExchange + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter @@ -140,11 +136,10 @@ CometNativeColumnarToRow : : : +- CometBroadcastHashJoin : : : :- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometSubqueryBroadcast + : : : : +- CometBroadcastExchange + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter @@ -167,11 +162,10 @@ CometNativeColumnarToRow : : +- CometBroadcastHashJoin : : :- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometSubqueryBroadcast + : : : +- CometBroadcastExchange + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- CometBroadcastExchange : : +- CometProject : : +- CometFilter @@ -185,4 +179,4 @@ CometNativeColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.web_returns -Comet accelerated 167 out of 179 eligible operators (93%). Final plan contains 7 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 173 out of 179 eligible operators (96%). Final plan contains 1 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q77.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q77.native_datafusion/extended.txt index 835f7a449a..8805a2e6c2 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q77.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q77.native_datafusion/extended.txt @@ -17,12 +17,11 @@ TakeOrderedAndProject : : : +- CometBroadcastHashJoin : : : :- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometSubqueryBroadcast + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter @@ -40,12 +39,11 @@ TakeOrderedAndProject : : +- CometBroadcastHashJoin : : :- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.store_returns - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometSubqueryBroadcast + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- CometBroadcastExchange : : +- CometProject : : +- CometFilter @@ -63,12 +61,11 @@ TakeOrderedAndProject : : +- CometProject : : +- CometBroadcastHashJoin : : :- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometSubqueryBroadcast + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- CometBroadcastExchange : : +- CometProject : : +- CometFilter @@ -80,12 +77,11 @@ TakeOrderedAndProject : +- CometProject : +- CometBroadcastHashJoin : :- CometNativeScan parquet spark_catalog.default.catalog_returns - : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometSubqueryBroadcast + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim : +- CometBroadcastExchange : +- CometProject : +- CometFilter @@ -102,12 +98,11 @@ TakeOrderedAndProject : : +- CometBroadcastHashJoin : : :- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometSubqueryBroadcast + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- CometBroadcastExchange : : +- CometProject : : +- CometFilter @@ -125,12 +120,11 @@ TakeOrderedAndProject : +- CometBroadcastHashJoin : :- CometFilter : : +- CometNativeScan parquet spark_catalog.default.web_returns - : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometSubqueryBroadcast + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim : +- CometBroadcastExchange : +- CometProject : +- CometFilter @@ -139,4 +133,4 @@ TakeOrderedAndProject +- CometFilter +- CometNativeScan parquet spark_catalog.default.web_page -Comet accelerated 109 out of 129 eligible operators (84%). Final plan contains 11 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 115 out of 129 eligible operators (89%). Final plan contains 5 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q78.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q78.native_datafusion/extended.txt index ebfdcc3c62..8426b9653c 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q78.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q78.native_datafusion/extended.txt @@ -17,11 +17,10 @@ TakeOrderedAndProject : : : : +- CometExchange : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometSubqueryBroadcast + : : : : +- CometBroadcastExchange + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- CometSort : : : +- CometExchange : : : +- CometProject @@ -44,11 +43,10 @@ TakeOrderedAndProject : : : +- CometExchange : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometSubqueryBroadcast + : : : +- CometBroadcastExchange + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- CometSort : : +- CometExchange : : +- CometProject @@ -71,11 +69,10 @@ TakeOrderedAndProject : : +- CometExchange : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometSubqueryBroadcast + : : +- CometBroadcastExchange + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim : +- CometSort : +- CometExchange : +- CometProject @@ -85,4 +82,4 @@ TakeOrderedAndProject +- CometFilter +- CometNativeScan parquet spark_catalog.default.date_dim -Comet accelerated 74 out of 82 eligible operators (90%). Final plan contains 4 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 77 out of 82 eligible operators (93%). Final plan contains 1 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q79.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q79.native_datafusion/extended.txt index 526b86f475..5af6449428 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q79.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q79.native_datafusion/extended.txt @@ -13,12 +13,11 @@ CometNativeColumnarToRow : : : +- CometBroadcastHashJoin : : : :- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometSubqueryBroadcast + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter @@ -36,4 +35,4 @@ CometNativeColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.customer -Comet accelerated 33 out of 35 eligible operators (94%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 34 out of 35 eligible operators (97%). Final plan contains 1 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q8.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q8.native_datafusion/extended.txt index cb04fc12c8..209a4e2bc0 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q8.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q8.native_datafusion/extended.txt @@ -11,12 +11,11 @@ CometNativeColumnarToRow : : +- CometBroadcastHashJoin : : :- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometSubqueryBroadcast + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- CometBroadcastExchange : : +- CometProject : : +- CometFilter @@ -49,4 +48,4 @@ CometNativeColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.customer -Comet accelerated 46 out of 48 eligible operators (95%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 47 out of 48 eligible operators (97%). Final plan contains 1 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q80.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q80.native_datafusion/extended.txt index 003be45ee5..27e32a1602 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q80.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q80.native_datafusion/extended.txt @@ -22,12 +22,11 @@ CometNativeColumnarToRow : : : : : : +- CometExchange : : : : : : +- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometNativeColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : +- CometSubqueryBroadcast + : : : : : : +- CometBroadcastExchange + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : : +- CometSort : : : : : +- CometExchange : : : : : +- CometProject @@ -66,12 +65,11 @@ CometNativeColumnarToRow : : : : : : +- CometExchange : : : : : : +- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometNativeColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : +- CometSubqueryBroadcast + : : : : : : +- CometBroadcastExchange + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : : +- CometSort : : : : : +- CometExchange : : : : : +- CometProject @@ -110,12 +108,11 @@ CometNativeColumnarToRow : : : : : +- CometExchange : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- CometSubqueryBroadcast + : : : : : +- CometBroadcastExchange + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : +- CometSort : : : : +- CometExchange : : : : +- CometProject @@ -138,4 +135,4 @@ CometNativeColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.promotion -Comet accelerated 129 out of 135 eligible operators (95%). Final plan contains 4 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 132 out of 135 eligible operators (97%). Final plan contains 1 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q81.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q81.native_datafusion/extended.txt index 949d50a867..6d9aa9697d 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q81.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q81.native_datafusion/extended.txt @@ -16,12 +16,11 @@ CometNativeColumnarToRow : : : : +- CometBroadcastHashJoin : : : : :- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_returns - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- CometSubqueryBroadcast + : : : : : +- CometBroadcastExchange + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : +- CometBroadcastExchange : : : : +- CometProject : : : : +- CometFilter @@ -44,12 +43,11 @@ CometNativeColumnarToRow : : : +- CometBroadcastHashJoin : : : :- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.catalog_returns - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometSubqueryBroadcast + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter @@ -67,4 +65,4 @@ CometNativeColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.customer_address -Comet accelerated 61 out of 65 eligible operators (93%). Final plan contains 3 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 63 out of 65 eligible operators (96%). Final plan contains 1 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q82.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q82.native_datafusion/extended.txt index 30fca82ec8..e7b75e3a55 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q82.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q82.native_datafusion/extended.txt @@ -17,12 +17,11 @@ CometNativeColumnarToRow : : +- CometProject : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.inventory - : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometSubqueryBroadcast + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim : +- CometBroadcastExchange : +- CometProject : +- CometFilter @@ -31,4 +30,4 @@ CometNativeColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.store_sales -Comet accelerated 28 out of 30 eligible operators (93%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 29 out of 30 eligible operators (96%). Final plan contains 1 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q83.ansi.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q83.ansi.native_datafusion/extended.txt index 3a16430014..e4d1992492 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q83.ansi.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q83.ansi.native_datafusion/extended.txt @@ -13,21 +13,20 @@ CometNativeColumnarToRow : : : +- CometBroadcastHashJoin : : : :- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.store_returns - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometNativeScan parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometSubqueryBroadcast + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter @@ -55,21 +54,20 @@ CometNativeColumnarToRow : : +- CometBroadcastHashJoin : : :- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.catalog_returns - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometSubqueryBroadcast + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- CometBroadcastExchange : : +- CometProject : : +- CometFilter @@ -97,21 +95,20 @@ CometNativeColumnarToRow : +- CometBroadcastHashJoin : :- CometFilter : : +- CometNativeScan parquet spark_catalog.default.web_returns - : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometNativeScan parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometSubqueryBroadcast + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim : +- CometBroadcastExchange : +- CometProject : +- CometFilter @@ -130,4 +127,4 @@ CometNativeColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.date_dim -Comet accelerated 121 out of 127 eligible operators (95%). Final plan contains 4 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 124 out of 127 eligible operators (97%). Final plan contains 1 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q85.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q85.native_datafusion/extended.txt index 06537a5eec..13506e196f 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q85.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q85.native_datafusion/extended.txt @@ -20,12 +20,11 @@ CometNativeColumnarToRow : : : : : : :- CometBroadcastExchange : : : : : : : +- CometFilter : : : : : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : : : : : +- SubqueryBroadcast - : : : : : : : +- BroadcastExchange - : : : : : : : +- CometNativeColumnarToRow - : : : : : : : +- CometProject - : : : : : : : +- CometFilter - : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : : +- CometSubqueryBroadcast + : : : : : : : +- CometBroadcastExchange + : : : : : : : +- CometProject + : : : : : : : +- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : : : +- CometProject : : : : : : +- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.web_returns @@ -53,4 +52,4 @@ CometNativeColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.reason -Comet accelerated 50 out of 52 eligible operators (96%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 51 out of 52 eligible operators (98%). Final plan contains 1 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q86.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q86.native_datafusion/extended.txt index c9117e2ae0..bddf80b4d0 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q86.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q86.native_datafusion/extended.txt @@ -14,12 +14,11 @@ TakeOrderedAndProject : +- CometBroadcastHashJoin : :- CometFilter : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometSubqueryBroadcast + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim : +- CometBroadcastExchange : +- CometProject : +- CometFilter @@ -29,4 +28,4 @@ TakeOrderedAndProject +- CometFilter +- CometNativeScan parquet spark_catalog.default.item -Comet accelerated 23 out of 28 eligible operators (82%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 24 out of 28 eligible operators (85%). Final plan contains 1 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q87.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q87.native_datafusion/extended.txt index 6676ec7a4b..5c80165c1f 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q87.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q87.native_datafusion/extended.txt @@ -15,12 +15,11 @@ HashAggregate : : : +- CometBroadcastHashJoin : : : :- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometSubqueryBroadcast + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter @@ -40,12 +39,11 @@ HashAggregate : : +- CometBroadcastHashJoin : : :- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometSubqueryBroadcast + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- CometBroadcastExchange : : +- CometProject : : +- CometFilter @@ -65,12 +63,11 @@ HashAggregate : +- CometBroadcastHashJoin : :- CometFilter : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometSubqueryBroadcast + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim : +- CometBroadcastExchange : +- CometProject : +- CometFilter @@ -80,4 +77,4 @@ HashAggregate +- CometFilter +- CometNativeScan parquet spark_catalog.default.customer -Comet accelerated 61 out of 74 eligible operators (82%). Final plan contains 7 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 64 out of 74 eligible operators (86%). Final plan contains 4 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q89.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q89.native_datafusion/extended.txt index 936cde052f..9a8a9bbba4 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q89.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q89.native_datafusion/extended.txt @@ -20,12 +20,11 @@ TakeOrderedAndProject : : +- CometBroadcastExchange : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometSubqueryBroadcast + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim : +- CometBroadcastExchange : +- CometProject : +- CometFilter @@ -34,4 +33,4 @@ TakeOrderedAndProject +- CometFilter +- CometNativeScan parquet spark_catalog.default.store -Comet accelerated 27 out of 33 eligible operators (81%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 28 out of 33 eligible operators (84%). Final plan contains 1 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q91.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q91.native_datafusion/extended.txt index 28bc6eab2f..9b191f8492 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q91.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q91.native_datafusion/extended.txt @@ -22,12 +22,11 @@ CometNativeColumnarToRow : : : : : +- CometBroadcastExchange : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_returns - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- CometSubqueryBroadcast + : : : : : +- CometBroadcastExchange + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : +- CometBroadcastExchange : : : : +- CometProject : : : : +- CometFilter @@ -48,4 +47,4 @@ CometNativeColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.household_demographics -Comet accelerated 45 out of 47 eligible operators (95%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 46 out of 47 eligible operators (97%). Final plan contains 1 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q92.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q92.native_datafusion/extended.txt index 6c5b57b3a1..40926fafac 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q92.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q92.native_datafusion/extended.txt @@ -10,12 +10,11 @@ CometNativeColumnarToRow : : +- CometBroadcastHashJoin : : :- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometSubqueryBroadcast + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- CometBroadcastExchange : : +- CometProject : : +- CometFilter @@ -29,12 +28,11 @@ CometNativeColumnarToRow : +- CometBroadcastHashJoin : :- CometFilter : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometSubqueryBroadcast + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim : +- CometBroadcastExchange : +- CometProject : +- CometFilter @@ -44,4 +42,4 @@ CometNativeColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.date_dim -Comet accelerated 38 out of 42 eligible operators (90%). Final plan contains 3 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 40 out of 42 eligible operators (95%). Final plan contains 1 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q97.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q97.native_datafusion/extended.txt index b63930a880..74163dff2b 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q97.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q97.native_datafusion/extended.txt @@ -11,12 +11,11 @@ CometNativeColumnarToRow : +- CometProject : +- CometBroadcastHashJoin : :- CometNativeScan parquet spark_catalog.default.store_sales - : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometSubqueryBroadcast + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim : +- CometBroadcastExchange : +- CometProject : +- CometFilter @@ -28,15 +27,14 @@ CometNativeColumnarToRow +- CometProject +- CometBroadcastHashJoin :- CometNativeScan parquet spark_catalog.default.catalog_sales - : +- SubqueryBroadcast - : +- BroadcastExchange - : +- CometNativeColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometSubqueryBroadcast + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.date_dim +- CometBroadcastExchange +- CometProject +- CometFilter +- CometNativeScan parquet spark_catalog.default.date_dim -Comet accelerated 33 out of 37 eligible operators (89%). Final plan contains 3 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 35 out of 37 eligible operators (94%). Final plan contains 1 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q98.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q98.native_datafusion/extended.txt index 6cc173560b..f1fc5ff9a3 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q98.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q98.native_datafusion/extended.txt @@ -16,12 +16,11 @@ CometNativeColumnarToRow : +- CometBroadcastHashJoin : :- CometFilter : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometSubqueryBroadcast + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim : +- CometBroadcastExchange : +- CometProject : +- CometFilter @@ -31,4 +30,4 @@ CometNativeColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.date_dim -Comet accelerated 25 out of 29 eligible operators (86%). Final plan contains 3 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 26 out of 29 eligible operators (89%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q1.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q1.native_datafusion/extended.txt index 935c103614..5875fa3a5e 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q1.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q1.native_datafusion/extended.txt @@ -14,12 +14,11 @@ CometNativeColumnarToRow : : : +- CometBroadcastHashJoin : : : :- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.store_returns - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometSubqueryBroadcast + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter @@ -36,12 +35,11 @@ CometNativeColumnarToRow : : +- CometBroadcastHashJoin : : :- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.store_returns - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometSubqueryBroadcast + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- CometBroadcastExchange : : +- CometProject : : +- CometFilter @@ -55,4 +53,4 @@ CometNativeColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.customer -Comet accelerated 49 out of 53 eligible operators (92%). Final plan contains 3 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 51 out of 53 eligible operators (96%). Final plan contains 1 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q10.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q10.native_datafusion/extended.txt index 2785c3d0b1..2e79f6f464 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q10.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q10.native_datafusion/extended.txt @@ -19,12 +19,11 @@ TakeOrderedAndProject : : : : +- CometProject : : : : +- CometBroadcastHashJoin : : : : :- CometNativeScan parquet spark_catalog.default.store_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- CometSubqueryBroadcast + : : : : : +- CometBroadcastExchange + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : +- CometBroadcastExchange : : : : +- CometProject : : : : +- CometFilter @@ -34,12 +33,11 @@ TakeOrderedAndProject : : : +- CometProject : : : +- CometBroadcastHashJoin : : : :- CometNativeScan parquet spark_catalog.default.web_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometSubqueryBroadcast + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter @@ -49,12 +47,11 @@ TakeOrderedAndProject : : +- CometProject : : +- CometBroadcastHashJoin : : :- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometSubqueryBroadcast + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- CometBroadcastExchange : : +- CometProject : : +- CometFilter @@ -70,4 +67,4 @@ TakeOrderedAndProject +- CometFilter +- CometNativeScan parquet spark_catalog.default.customer_demographics -Comet accelerated 41 out of 62 eligible operators (66%). Final plan contains 9 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 44 out of 62 eligible operators (70%). Final plan contains 6 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q11.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q11.native_datafusion/extended.txt index 6b081dd35b..edda694b73 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q11.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q11.native_datafusion/extended.txt @@ -20,11 +20,10 @@ CometNativeColumnarToRow : : : : +- CometBroadcastExchange : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometSubqueryBroadcast + : : : : +- CometBroadcastExchange + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- CometBroadcastExchange : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -42,11 +41,10 @@ CometNativeColumnarToRow : : : +- CometBroadcastExchange : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometSubqueryBroadcast + : : : +- CometBroadcastExchange + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- CometBroadcastExchange : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -65,11 +63,10 @@ CometNativeColumnarToRow : : +- CometBroadcastExchange : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometSubqueryBroadcast + : : +- CometBroadcastExchange + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim : +- CometBroadcastExchange : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -87,13 +84,12 @@ CometNativeColumnarToRow : +- CometBroadcastExchange : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.web_sales - : +- SubqueryBroadcast - : +- BroadcastExchange - : +- CometNativeColumnarToRow - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometSubqueryBroadcast + : +- CometBroadcastExchange + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.date_dim +- CometBroadcastExchange +- CometFilter +- CometNativeScan parquet spark_catalog.default.date_dim -Comet accelerated 84 out of 92 eligible operators (91%). Final plan contains 5 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 88 out of 92 eligible operators (95%). Final plan contains 1 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q12.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q12.native_datafusion/extended.txt index a8c2531a38..a6cc526735 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q12.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q12.native_datafusion/extended.txt @@ -13,12 +13,11 @@ TakeOrderedAndProject : +- CometBroadcastHashJoin : :- CometFilter : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometSubqueryBroadcast + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim : +- CometBroadcastExchange : +- CometProject : +- CometFilter @@ -28,4 +27,4 @@ TakeOrderedAndProject +- CometFilter +- CometNativeScan parquet spark_catalog.default.date_dim -Comet accelerated 22 out of 27 eligible operators (81%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 23 out of 27 eligible operators (85%). Final plan contains 1 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q13.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q13.native_datafusion/extended.txt index 42ea1c4966..33c2dd1bbc 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q13.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q13.native_datafusion/extended.txt @@ -14,12 +14,11 @@ CometNativeColumnarToRow : : : : +- CometBroadcastHashJoin : : : : :- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- CometSubqueryBroadcast + : : : : : +- CometBroadcastExchange + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : +- CometBroadcastExchange : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.store @@ -39,4 +38,4 @@ CometNativeColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.household_demographics -Comet accelerated 36 out of 38 eligible operators (94%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 37 out of 38 eligible operators (97%). Final plan contains 1 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14a.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14a.native_datafusion/extended.txt index 0dcd47f57f..d2a44e9c52 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14a.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14a.native_datafusion/extended.txt @@ -16,12 +16,11 @@ CometNativeColumnarToRow : : :- CometProject : : : +- CometBroadcastHashJoin : : : :- CometNativeScan parquet spark_catalog.default.store_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometSubqueryBroadcast + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter @@ -29,12 +28,11 @@ CometNativeColumnarToRow : : :- CometProject : : : +- CometBroadcastHashJoin : : : :- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometSubqueryBroadcast + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter @@ -42,12 +40,11 @@ CometNativeColumnarToRow : : +- CometProject : : +- CometBroadcastHashJoin : : :- CometNativeScan parquet spark_catalog.default.web_sales - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometSubqueryBroadcast + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- CometBroadcastExchange : : +- CometProject : : +- CometFilter @@ -62,12 +59,11 @@ CometNativeColumnarToRow : : :- CometBroadcastHashJoin : : : :- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometSubqueryBroadcast + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometBroadcastHashJoin @@ -84,12 +80,11 @@ CometNativeColumnarToRow : : : : : +- CometBroadcastHashJoin : : : : : :- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometNativeColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : +- CometSubqueryBroadcast + : : : : : : +- CometBroadcastExchange + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : : +- CometBroadcastExchange : : : : : +- CometBroadcastHashJoin : : : : : :- CometFilter @@ -101,12 +96,11 @@ CometNativeColumnarToRow : : : : : : +- CometBroadcastHashJoin : : : : : : :- CometFilter : : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : : : : +- SubqueryBroadcast - : : : : : : : +- BroadcastExchange - : : : : : : : +- CometNativeColumnarToRow - : : : : : : : +- CometProject - : : : : : : : +- CometFilter - : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : : +- CometSubqueryBroadcast + : : : : : : : +- CometBroadcastExchange + : : : : : : : +- CometProject + : : : : : : : +- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : : : +- CometBroadcastExchange : : : : : : +- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.item @@ -125,12 +119,11 @@ CometNativeColumnarToRow : : : : +- CometBroadcastHashJoin : : : : :- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- CometSubqueryBroadcast + : : : : : +- CometBroadcastExchange + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : +- CometBroadcastExchange : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.item @@ -158,12 +151,11 @@ CometNativeColumnarToRow : : : : +- CometBroadcastHashJoin : : : : :- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- CometSubqueryBroadcast + : : : : : +- CometBroadcastExchange + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : +- CometBroadcastExchange : : : : +- CometBroadcastHashJoin : : : : :- CometFilter @@ -175,12 +167,11 @@ CometNativeColumnarToRow : : : : : +- CometBroadcastHashJoin : : : : : :- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometNativeColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : +- CometSubqueryBroadcast + : : : : : : +- CometBroadcastExchange + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : : +- CometBroadcastExchange : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.item @@ -199,12 +190,11 @@ CometNativeColumnarToRow : : : +- CometBroadcastHashJoin : : : :- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometSubqueryBroadcast + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- CometBroadcastExchange : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.item @@ -229,12 +219,11 @@ CometNativeColumnarToRow : : :- CometBroadcastHashJoin : : : :- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometSubqueryBroadcast + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometBroadcastHashJoin @@ -251,12 +240,11 @@ CometNativeColumnarToRow : : : : : +- CometBroadcastHashJoin : : : : : :- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometNativeColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : +- CometSubqueryBroadcast + : : : : : : +- CometBroadcastExchange + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : : +- CometBroadcastExchange : : : : : +- CometBroadcastHashJoin : : : : : :- CometFilter @@ -268,12 +256,11 @@ CometNativeColumnarToRow : : : : : : +- CometBroadcastHashJoin : : : : : : :- CometFilter : : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : : : : +- SubqueryBroadcast - : : : : : : : +- BroadcastExchange - : : : : : : : +- CometNativeColumnarToRow - : : : : : : : +- CometProject - : : : : : : : +- CometFilter - : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : : +- CometSubqueryBroadcast + : : : : : : : +- CometBroadcastExchange + : : : : : : : +- CometProject + : : : : : : : +- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : : : +- CometBroadcastExchange : : : : : : +- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.item @@ -292,12 +279,11 @@ CometNativeColumnarToRow : : : : +- CometBroadcastHashJoin : : : : :- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- CometSubqueryBroadcast + : : : : : +- CometBroadcastExchange + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : +- CometBroadcastExchange : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.item @@ -325,12 +311,11 @@ CometNativeColumnarToRow : : : : +- CometBroadcastHashJoin : : : : :- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- CometSubqueryBroadcast + : : : : : +- CometBroadcastExchange + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : +- CometBroadcastExchange : : : : +- CometBroadcastHashJoin : : : : :- CometFilter @@ -342,12 +327,11 @@ CometNativeColumnarToRow : : : : : +- CometBroadcastHashJoin : : : : : :- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometNativeColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : +- CometSubqueryBroadcast + : : : : : : +- CometBroadcastExchange + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : : +- CometBroadcastExchange : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.item @@ -366,12 +350,11 @@ CometNativeColumnarToRow : : : +- CometBroadcastHashJoin : : : :- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometSubqueryBroadcast + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- CometBroadcastExchange : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.item @@ -396,12 +379,11 @@ CometNativeColumnarToRow : :- CometBroadcastHashJoin : : :- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometSubqueryBroadcast + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- CometBroadcastExchange : : +- CometProject : : +- CometBroadcastHashJoin @@ -418,12 +400,11 @@ CometNativeColumnarToRow : : : : +- CometBroadcastHashJoin : : : : :- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- CometSubqueryBroadcast + : : : : : +- CometBroadcastExchange + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : +- CometBroadcastExchange : : : : +- CometBroadcastHashJoin : : : : :- CometFilter @@ -435,12 +416,11 @@ CometNativeColumnarToRow : : : : : +- CometBroadcastHashJoin : : : : : :- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometNativeColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : +- CometSubqueryBroadcast + : : : : : : +- CometBroadcastExchange + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : : +- CometBroadcastExchange : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.item @@ -459,12 +439,11 @@ CometNativeColumnarToRow : : : +- CometBroadcastHashJoin : : : :- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometSubqueryBroadcast + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- CometBroadcastExchange : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.item @@ -492,12 +471,11 @@ CometNativeColumnarToRow : : : +- CometBroadcastHashJoin : : : :- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometSubqueryBroadcast + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- CometBroadcastExchange : : : +- CometBroadcastHashJoin : : : :- CometFilter @@ -509,12 +487,11 @@ CometNativeColumnarToRow : : : : +- CometBroadcastHashJoin : : : : :- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- CometSubqueryBroadcast + : : : : : +- CometBroadcastExchange + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : +- CometBroadcastExchange : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.item @@ -533,12 +510,11 @@ CometNativeColumnarToRow : : +- CometBroadcastHashJoin : : :- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometSubqueryBroadcast + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- CometBroadcastExchange : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.item @@ -551,4 +527,4 @@ CometNativeColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.date_dim -Comet accelerated 475 out of 526 eligible operators (90%). Final plan contains 26 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 499 out of 526 eligible operators (94%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14b.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14b.native_datafusion/extended.txt index 50139381f2..2d20c40e4b 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14b.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14b.native_datafusion/extended.txt @@ -11,12 +11,11 @@ CometNativeColumnarToRow : : :- CometProject : : : +- CometBroadcastHashJoin : : : :- CometNativeScan parquet spark_catalog.default.store_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometSubqueryBroadcast + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter @@ -24,12 +23,11 @@ CometNativeColumnarToRow : : :- CometProject : : : +- CometBroadcastHashJoin : : : :- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometSubqueryBroadcast + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter @@ -37,12 +35,11 @@ CometNativeColumnarToRow : : +- CometProject : : +- CometBroadcastHashJoin : : :- CometNativeScan parquet spark_catalog.default.web_sales - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometSubqueryBroadcast + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- CometBroadcastExchange : : +- CometProject : : +- CometFilter @@ -57,17 +54,16 @@ CometNativeColumnarToRow : : :- CometBroadcastHashJoin : : : :- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : : +- Subquery - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometSubqueryBroadcast + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : : +- Subquery + : : : : : +- CometNativeColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometBroadcastHashJoin @@ -84,12 +80,11 @@ CometNativeColumnarToRow : : : : : +- CometBroadcastHashJoin : : : : : :- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometNativeColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : +- CometSubqueryBroadcast + : : : : : : +- CometBroadcastExchange + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : : +- CometBroadcastExchange : : : : : +- CometBroadcastHashJoin : : : : : :- CometFilter @@ -101,12 +96,11 @@ CometNativeColumnarToRow : : : : : : +- CometBroadcastHashJoin : : : : : : :- CometFilter : : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : : : : +- SubqueryBroadcast - : : : : : : : +- BroadcastExchange - : : : : : : : +- CometNativeColumnarToRow - : : : : : : : +- CometProject - : : : : : : : +- CometFilter - : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : : +- CometSubqueryBroadcast + : : : : : : : +- CometBroadcastExchange + : : : : : : : +- CometProject + : : : : : : : +- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : : : +- CometBroadcastExchange : : : : : : +- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.item @@ -125,12 +119,11 @@ CometNativeColumnarToRow : : : : +- CometBroadcastHashJoin : : : : :- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- CometSubqueryBroadcast + : : : : : +- CometBroadcastExchange + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : +- CometBroadcastExchange : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.item @@ -158,12 +151,11 @@ CometNativeColumnarToRow : : : : +- CometBroadcastHashJoin : : : : :- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- CometSubqueryBroadcast + : : : : : +- CometBroadcastExchange + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : +- CometBroadcastExchange : : : : +- CometBroadcastHashJoin : : : : :- CometFilter @@ -175,12 +167,11 @@ CometNativeColumnarToRow : : : : : +- CometBroadcastHashJoin : : : : : :- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometNativeColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : +- CometSubqueryBroadcast + : : : : : : +- CometBroadcastExchange + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : : +- CometBroadcastExchange : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.item @@ -199,12 +190,11 @@ CometNativeColumnarToRow : : : +- CometBroadcastHashJoin : : : :- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometSubqueryBroadcast + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- CometBroadcastExchange : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.item @@ -215,7 +205,11 @@ CometNativeColumnarToRow : +- CometBroadcastExchange : +- CometProject : +- CometFilter - : : +- ReusedSubquery + : : +- Subquery + : : +- CometNativeColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim : +- CometNativeScan parquet spark_catalog.default.date_dim +- CometBroadcastExchange +- CometFilter @@ -230,17 +224,16 @@ CometNativeColumnarToRow : :- CometBroadcastHashJoin : : :- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : : +- Subquery - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometSubqueryBroadcast + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : : +- Subquery + : : : : +- CometNativeColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- CometBroadcastExchange : : +- CometProject : : +- CometBroadcastHashJoin @@ -257,12 +250,11 @@ CometNativeColumnarToRow : : : : +- CometBroadcastHashJoin : : : : :- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- CometSubqueryBroadcast + : : : : : +- CometBroadcastExchange + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : +- CometBroadcastExchange : : : : +- CometBroadcastHashJoin : : : : :- CometFilter @@ -274,12 +266,11 @@ CometNativeColumnarToRow : : : : : +- CometBroadcastHashJoin : : : : : :- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometNativeColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : +- CometSubqueryBroadcast + : : : : : : +- CometBroadcastExchange + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : : +- CometBroadcastExchange : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.item @@ -298,12 +289,11 @@ CometNativeColumnarToRow : : : +- CometBroadcastHashJoin : : : :- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometSubqueryBroadcast + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- CometBroadcastExchange : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.item @@ -331,12 +321,11 @@ CometNativeColumnarToRow : : : +- CometBroadcastHashJoin : : : :- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometSubqueryBroadcast + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- CometBroadcastExchange : : : +- CometBroadcastHashJoin : : : :- CometFilter @@ -348,12 +337,11 @@ CometNativeColumnarToRow : : : : +- CometBroadcastHashJoin : : : : :- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- CometSubqueryBroadcast + : : : : : +- CometBroadcastExchange + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : +- CometBroadcastExchange : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.item @@ -372,12 +360,11 @@ CometNativeColumnarToRow : : +- CometBroadcastHashJoin : : :- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometSubqueryBroadcast + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- CometBroadcastExchange : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.item @@ -388,7 +375,11 @@ CometNativeColumnarToRow +- CometBroadcastExchange +- CometProject +- CometFilter - : +- ReusedSubquery + : +- Subquery + : +- CometNativeColumnarToRow + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.date_dim +- CometNativeScan parquet spark_catalog.default.date_dim -Comet accelerated 331 out of 371 eligible operators (89%). Final plan contains 21 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 354 out of 377 eligible operators (93%). Final plan contains 6 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q15.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q15.native_datafusion/extended.txt index 42432bdc12..da1e91c969 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q15.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q15.native_datafusion/extended.txt @@ -11,12 +11,11 @@ CometNativeColumnarToRow : : +- CometBroadcastHashJoin : : :- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometSubqueryBroadcast + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- CometBroadcastExchange : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.customer @@ -29,4 +28,4 @@ CometNativeColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.date_dim -Comet accelerated 26 out of 28 eligible operators (92%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 27 out of 28 eligible operators (96%). Final plan contains 1 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q17.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q17.native_datafusion/extended.txt index f58c012650..2650cd8f1a 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q17.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q17.native_datafusion/extended.txt @@ -19,30 +19,27 @@ CometNativeColumnarToRow : : : : : : +- CometBroadcastHashJoin : : : : : : :- CometFilter : : : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : : : : : +- SubqueryBroadcast - : : : : : : : +- BroadcastExchange - : : : : : : : +- CometNativeColumnarToRow - : : : : : : : +- CometProject - : : : : : : : +- CometFilter - : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : : +- CometSubqueryBroadcast + : : : : : : : +- CometBroadcastExchange + : : : : : : : +- CometProject + : : : : : : : +- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : : : +- CometBroadcastExchange : : : : : : +- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.store_returns - : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometNativeColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : +- CometSubqueryBroadcast + : : : : : : +- CometBroadcastExchange + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : : +- CometBroadcastExchange : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- CometSubqueryBroadcast + : : : : : +- CometBroadcastExchange + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : +- CometBroadcastExchange : : : : +- CometProject : : : : +- CometFilter @@ -64,4 +61,4 @@ CometNativeColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.item -Comet accelerated 55 out of 61 eligible operators (90%). Final plan contains 4 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 58 out of 61 eligible operators (95%). Final plan contains 1 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q18.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q18.native_datafusion/extended.txt index b3485a3421..4040a49d5a 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q18.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q18.native_datafusion/extended.txt @@ -18,12 +18,11 @@ CometNativeColumnarToRow : : : : : +- CometBroadcastHashJoin : : : : : :- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometNativeColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : +- CometSubqueryBroadcast + : : : : : : +- CometBroadcastExchange + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : : +- CometBroadcastExchange : : : : : +- CometProject : : : : : +- CometFilter @@ -48,4 +47,4 @@ CometNativeColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.item -Comet accelerated 45 out of 47 eligible operators (95%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 46 out of 47 eligible operators (97%). Final plan contains 1 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q20.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q20.native_datafusion/extended.txt index c4f54f4289..24498162ca 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q20.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q20.native_datafusion/extended.txt @@ -13,12 +13,11 @@ TakeOrderedAndProject : +- CometBroadcastHashJoin : :- CometFilter : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometSubqueryBroadcast + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim : +- CometBroadcastExchange : +- CometProject : +- CometFilter @@ -28,4 +27,4 @@ TakeOrderedAndProject +- CometFilter +- CometNativeScan parquet spark_catalog.default.date_dim -Comet accelerated 22 out of 27 eligible operators (81%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 23 out of 27 eligible operators (85%). Final plan contains 1 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q21.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q21.native_datafusion/extended.txt index 8522eb4ab9..9cbd136255 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q21.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q21.native_datafusion/extended.txt @@ -12,11 +12,10 @@ CometNativeColumnarToRow : : +- CometBroadcastHashJoin : : :- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.inventory - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometSubqueryBroadcast + : : : +- CometBroadcastExchange + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- CometBroadcastExchange : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.warehouse @@ -28,4 +27,4 @@ CometNativeColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.date_dim -Comet accelerated 25 out of 27 eligible operators (92%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 26 out of 27 eligible operators (96%). Final plan contains 1 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q22.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q22.native_datafusion/extended.txt index 6b78d21629..36a0eb0c8a 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q22.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q22.native_datafusion/extended.txt @@ -12,12 +12,11 @@ CometNativeColumnarToRow : : +- CometBroadcastHashJoin : : :- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.inventory - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometSubqueryBroadcast + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- CometBroadcastExchange : : +- CometProject : : +- CometFilter @@ -30,4 +29,4 @@ CometNativeColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.warehouse -Comet accelerated 27 out of 29 eligible operators (93%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 28 out of 29 eligible operators (96%). Final plan contains 1 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q23a.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q23a.native_datafusion/extended.txt index 8a9990d536..3e46e4a329 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q23a.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q23a.native_datafusion/extended.txt @@ -12,12 +12,11 @@ CometNativeColumnarToRow : : : +- CometProject : : : +- CometBroadcastHashJoin : : : :- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometSubqueryBroadcast + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter @@ -30,12 +29,11 @@ CometNativeColumnarToRow : : : : +- CometBroadcastHashJoin : : : : :- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- CometSubqueryBroadcast + : : : : : +- CometBroadcastExchange + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : +- CometBroadcastExchange : : : : +- CometProject : : : : +- CometFilter @@ -60,12 +58,11 @@ CometNativeColumnarToRow : : : : +- CometBroadcastHashJoin : : : : :- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- CometSubqueryBroadcast + : : : : : +- CometBroadcastExchange + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : +- CometBroadcastExchange : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.customer @@ -97,12 +94,11 @@ CometNativeColumnarToRow : : +- CometProject : : +- CometBroadcastHashJoin : : :- CometNativeScan parquet spark_catalog.default.web_sales - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometSubqueryBroadcast + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- CometBroadcastExchange : : +- CometProject : : +- CometFilter @@ -115,12 +111,11 @@ CometNativeColumnarToRow : : : +- CometBroadcastHashJoin : : : :- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometSubqueryBroadcast + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter @@ -148,4 +143,4 @@ CometNativeColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.date_dim -Comet accelerated 130 out of 142 eligible operators (91%). Final plan contains 7 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 135 out of 142 eligible operators (95%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q23b.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q23b.native_datafusion/extended.txt index 52090c668e..79baaf1aa9 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q23b.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q23b.native_datafusion/extended.txt @@ -15,12 +15,11 @@ CometNativeColumnarToRow : : : : +- CometBroadcastHashJoin : : : : :- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- CometSubqueryBroadcast + : : : : : +- CometBroadcastExchange + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : +- CometBroadcastExchange : : : : +- CometProject : : : : +- CometFilter @@ -33,12 +32,11 @@ CometNativeColumnarToRow : : : : : +- CometBroadcastHashJoin : : : : : :- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometNativeColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : +- CometSubqueryBroadcast + : : : : : : +- CometBroadcastExchange + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : : +- CometBroadcastExchange : : : : : +- CometProject : : : : : +- CometFilter @@ -63,12 +61,11 @@ CometNativeColumnarToRow : : : : : +- CometBroadcastHashJoin : : : : : :- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometNativeColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : +- CometSubqueryBroadcast + : : : : : : +- CometBroadcastExchange + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : : +- CometBroadcastExchange : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.customer @@ -127,12 +124,11 @@ CometNativeColumnarToRow : : : +- CometBroadcastHashJoin : : : :- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometSubqueryBroadcast + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter @@ -145,12 +141,11 @@ CometNativeColumnarToRow : : : : +- CometBroadcastHashJoin : : : : :- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- CometSubqueryBroadcast + : : : : : +- CometBroadcastExchange + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : +- CometBroadcastExchange : : : : +- CometProject : : : : +- CometFilter @@ -200,4 +195,4 @@ CometNativeColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.date_dim -Comet accelerated 180 out of 194 eligible operators (92%). Final plan contains 7 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 185 out of 194 eligible operators (95%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q25.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q25.native_datafusion/extended.txt index f58c012650..2650cd8f1a 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q25.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q25.native_datafusion/extended.txt @@ -19,30 +19,27 @@ CometNativeColumnarToRow : : : : : : +- CometBroadcastHashJoin : : : : : : :- CometFilter : : : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : : : : : +- SubqueryBroadcast - : : : : : : : +- BroadcastExchange - : : : : : : : +- CometNativeColumnarToRow - : : : : : : : +- CometProject - : : : : : : : +- CometFilter - : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : : +- CometSubqueryBroadcast + : : : : : : : +- CometBroadcastExchange + : : : : : : : +- CometProject + : : : : : : : +- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : : : +- CometBroadcastExchange : : : : : : +- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.store_returns - : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometNativeColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : +- CometSubqueryBroadcast + : : : : : : +- CometBroadcastExchange + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : : +- CometBroadcastExchange : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- CometSubqueryBroadcast + : : : : : +- CometBroadcastExchange + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : +- CometBroadcastExchange : : : : +- CometProject : : : : +- CometFilter @@ -64,4 +61,4 @@ CometNativeColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.item -Comet accelerated 55 out of 61 eligible operators (90%). Final plan contains 4 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 58 out of 61 eligible operators (95%). Final plan contains 1 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q26.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q26.native_datafusion/extended.txt index 71fd3bbe6f..2f7fe7b668 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q26.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q26.native_datafusion/extended.txt @@ -13,12 +13,11 @@ CometNativeColumnarToRow : : : +- CometBroadcastHashJoin : : : :- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometSubqueryBroadcast + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter @@ -36,4 +35,4 @@ CometNativeColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.promotion -Comet accelerated 33 out of 35 eligible operators (94%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 34 out of 35 eligible operators (97%). Final plan contains 1 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q27.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q27.native_datafusion/extended.txt index 15ab28f784..df27dd568f 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q27.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q27.native_datafusion/extended.txt @@ -14,12 +14,11 @@ CometNativeColumnarToRow : : : +- CometBroadcastHashJoin : : : :- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometSubqueryBroadcast + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter @@ -37,4 +36,4 @@ CometNativeColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.item -Comet accelerated 34 out of 36 eligible operators (94%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 35 out of 36 eligible operators (97%). Final plan contains 1 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q29.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q29.native_datafusion/extended.txt index f58c012650..2650cd8f1a 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q29.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q29.native_datafusion/extended.txt @@ -19,30 +19,27 @@ CometNativeColumnarToRow : : : : : : +- CometBroadcastHashJoin : : : : : : :- CometFilter : : : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : : : : : +- SubqueryBroadcast - : : : : : : : +- BroadcastExchange - : : : : : : : +- CometNativeColumnarToRow - : : : : : : : +- CometProject - : : : : : : : +- CometFilter - : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : : +- CometSubqueryBroadcast + : : : : : : : +- CometBroadcastExchange + : : : : : : : +- CometProject + : : : : : : : +- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : : : +- CometBroadcastExchange : : : : : : +- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.store_returns - : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometNativeColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : +- CometSubqueryBroadcast + : : : : : : +- CometBroadcastExchange + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : : +- CometBroadcastExchange : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- CometSubqueryBroadcast + : : : : : +- CometBroadcastExchange + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : +- CometBroadcastExchange : : : : +- CometProject : : : : +- CometFilter @@ -64,4 +61,4 @@ CometNativeColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.item -Comet accelerated 55 out of 61 eligible operators (90%). Final plan contains 4 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 58 out of 61 eligible operators (95%). Final plan contains 1 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q30.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q30.native_datafusion/extended.txt index 18d0a12a03..2a22518df4 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q30.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q30.native_datafusion/extended.txt @@ -16,12 +16,11 @@ CometNativeColumnarToRow : : : : +- CometBroadcastHashJoin : : : : :- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.web_returns - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- CometSubqueryBroadcast + : : : : : +- CometBroadcastExchange + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : +- CometBroadcastExchange : : : : +- CometProject : : : : +- CometFilter @@ -44,12 +43,11 @@ CometNativeColumnarToRow : : : +- CometBroadcastHashJoin : : : :- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.web_returns - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometSubqueryBroadcast + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter @@ -67,4 +65,4 @@ CometNativeColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.customer_address -Comet accelerated 61 out of 65 eligible operators (93%). Final plan contains 3 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 63 out of 65 eligible operators (96%). Final plan contains 1 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q31.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q31.native_datafusion/extended.txt index d747a5243e..bf4fb2aa77 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q31.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q31.native_datafusion/extended.txt @@ -18,11 +18,10 @@ CometNativeColumnarToRow : : : : : : +- CometBroadcastHashJoin : : : : : : :- CometFilter : : : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : : : : : +- SubqueryBroadcast - : : : : : : : +- BroadcastExchange - : : : : : : : +- CometNativeColumnarToRow - : : : : : : : +- CometFilter - : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : : +- CometSubqueryBroadcast + : : : : : : : +- CometBroadcastExchange + : : : : : : : +- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : : : +- CometBroadcastExchange : : : : : : +- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -39,11 +38,10 @@ CometNativeColumnarToRow : : : : : +- CometBroadcastHashJoin : : : : : :- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometNativeColumnarToRow - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : +- CometSubqueryBroadcast + : : : : : : +- CometBroadcastExchange + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : : +- CometBroadcastExchange : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -60,11 +58,10 @@ CometNativeColumnarToRow : : : : +- CometBroadcastHashJoin : : : : :- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- CometSubqueryBroadcast + : : : : : +- CometBroadcastExchange + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : +- CometBroadcastExchange : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -81,11 +78,10 @@ CometNativeColumnarToRow : : : +- CometBroadcastHashJoin : : : :- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometSubqueryBroadcast + : : : : +- CometBroadcastExchange + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- CometBroadcastExchange : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -102,11 +98,10 @@ CometNativeColumnarToRow : : +- CometBroadcastHashJoin : : :- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometSubqueryBroadcast + : : : +- CometBroadcastExchange + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- CometBroadcastExchange : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -123,11 +118,10 @@ CometNativeColumnarToRow : +- CometBroadcastHashJoin : :- CometFilter : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometSubqueryBroadcast + : : +- CometBroadcastExchange + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim : +- CometBroadcastExchange : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -135,4 +129,4 @@ CometNativeColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.customer_address -Comet accelerated 117 out of 129 eligible operators (90%). Final plan contains 7 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 123 out of 129 eligible operators (95%). Final plan contains 1 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q32.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q32.native_datafusion/extended.txt index 5d009dea2d..db07e648bc 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q32.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q32.native_datafusion/extended.txt @@ -10,12 +10,11 @@ CometNativeColumnarToRow : : +- CometBroadcastHashJoin : : :- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometSubqueryBroadcast + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- CometBroadcastExchange : : +- CometProject : : +- CometFilter @@ -29,12 +28,11 @@ CometNativeColumnarToRow : +- CometBroadcastHashJoin : :- CometFilter : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometSubqueryBroadcast + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim : +- CometBroadcastExchange : +- CometProject : +- CometFilter @@ -44,4 +42,4 @@ CometNativeColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.date_dim -Comet accelerated 38 out of 42 eligible operators (90%). Final plan contains 3 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 40 out of 42 eligible operators (95%). Final plan contains 1 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q33.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q33.native_datafusion/extended.txt index 6afaa68137..d64ba66952 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q33.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q33.native_datafusion/extended.txt @@ -15,12 +15,11 @@ CometNativeColumnarToRow : : : +- CometBroadcastHashJoin : : : :- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometSubqueryBroadcast + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter @@ -48,12 +47,11 @@ CometNativeColumnarToRow : : : +- CometBroadcastHashJoin : : : :- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometSubqueryBroadcast + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter @@ -81,12 +79,11 @@ CometNativeColumnarToRow : : +- CometBroadcastHashJoin : : :- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometSubqueryBroadcast + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- CometBroadcastExchange : : +- CometProject : : +- CometFilter @@ -104,4 +101,4 @@ CometNativeColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.item -Comet accelerated 95 out of 101 eligible operators (94%). Final plan contains 4 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 98 out of 101 eligible operators (97%). Final plan contains 1 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q34.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q34.native_datafusion/extended.txt index 1b61be2937..4c0ef9abcd 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q34.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q34.native_datafusion/extended.txt @@ -15,12 +15,11 @@ CometNativeColumnarToRow : : : +- CometBroadcastHashJoin : : : :- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometSubqueryBroadcast + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter @@ -38,4 +37,4 @@ CometNativeColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.customer -Comet accelerated 35 out of 37 eligible operators (94%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 36 out of 37 eligible operators (97%). Final plan contains 1 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q35.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q35.native_datafusion/extended.txt index 2785c3d0b1..2e79f6f464 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q35.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q35.native_datafusion/extended.txt @@ -19,12 +19,11 @@ TakeOrderedAndProject : : : : +- CometProject : : : : +- CometBroadcastHashJoin : : : : :- CometNativeScan parquet spark_catalog.default.store_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- CometSubqueryBroadcast + : : : : : +- CometBroadcastExchange + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : +- CometBroadcastExchange : : : : +- CometProject : : : : +- CometFilter @@ -34,12 +33,11 @@ TakeOrderedAndProject : : : +- CometProject : : : +- CometBroadcastHashJoin : : : :- CometNativeScan parquet spark_catalog.default.web_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometSubqueryBroadcast + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter @@ -49,12 +47,11 @@ TakeOrderedAndProject : : +- CometProject : : +- CometBroadcastHashJoin : : :- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometSubqueryBroadcast + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- CometBroadcastExchange : : +- CometProject : : +- CometFilter @@ -70,4 +67,4 @@ TakeOrderedAndProject +- CometFilter +- CometNativeScan parquet spark_catalog.default.customer_demographics -Comet accelerated 41 out of 62 eligible operators (66%). Final plan contains 9 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 44 out of 62 eligible operators (70%). Final plan contains 6 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q36.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q36.native_datafusion/extended.txt index 05ce793d8d..9480047251 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q36.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q36.native_datafusion/extended.txt @@ -16,12 +16,11 @@ TakeOrderedAndProject : : +- CometBroadcastHashJoin : : :- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometSubqueryBroadcast + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- CometBroadcastExchange : : +- CometProject : : +- CometFilter @@ -35,4 +34,4 @@ TakeOrderedAndProject +- CometFilter +- CometNativeScan parquet spark_catalog.default.store -Comet accelerated 29 out of 34 eligible operators (85%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 30 out of 34 eligible operators (88%). Final plan contains 1 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q37.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q37.native_datafusion/extended.txt index 16e3a70d9b..a83c218727 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q37.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q37.native_datafusion/extended.txt @@ -17,12 +17,11 @@ CometNativeColumnarToRow : : +- CometProject : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.inventory - : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometSubqueryBroadcast + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim : +- CometBroadcastExchange : +- CometProject : +- CometFilter @@ -31,4 +30,4 @@ CometNativeColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.catalog_sales -Comet accelerated 28 out of 30 eligible operators (93%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 29 out of 30 eligible operators (96%). Final plan contains 1 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q38.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q38.native_datafusion/extended.txt index 29422273c5..c0314d8fb0 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q38.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q38.native_datafusion/extended.txt @@ -14,12 +14,11 @@ CometNativeColumnarToRow : : : +- CometBroadcastHashJoin : : : :- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometSubqueryBroadcast + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter @@ -38,12 +37,11 @@ CometNativeColumnarToRow : : +- CometBroadcastHashJoin : : :- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometSubqueryBroadcast + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- CometBroadcastExchange : : +- CometProject : : +- CometFilter @@ -62,12 +60,11 @@ CometNativeColumnarToRow : +- CometBroadcastHashJoin : :- CometFilter : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometSubqueryBroadcast + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim : +- CometBroadcastExchange : +- CometProject : +- CometFilter @@ -77,4 +74,4 @@ CometNativeColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.customer -Comet accelerated 68 out of 74 eligible operators (91%). Final plan contains 4 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 71 out of 74 eligible operators (95%). Final plan contains 1 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q39a.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q39a.native_datafusion/extended.txt index eba1ff4dcd..67ff950c02 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q39a.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q39a.native_datafusion/extended.txt @@ -15,12 +15,11 @@ CometNativeColumnarToRow : : : +- CometBroadcastHashJoin : : : :- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.inventory - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometSubqueryBroadcast + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- CometBroadcastExchange : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.item @@ -45,12 +44,11 @@ CometNativeColumnarToRow : : +- CometBroadcastHashJoin : : :- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.inventory - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometSubqueryBroadcast + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- CometBroadcastExchange : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.item @@ -62,4 +60,4 @@ CometNativeColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.date_dim -Comet accelerated 56 out of 60 eligible operators (93%). Final plan contains 3 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 58 out of 60 eligible operators (96%). Final plan contains 1 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q39b.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q39b.native_datafusion/extended.txt index eba1ff4dcd..67ff950c02 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q39b.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q39b.native_datafusion/extended.txt @@ -15,12 +15,11 @@ CometNativeColumnarToRow : : : +- CometBroadcastHashJoin : : : :- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.inventory - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometSubqueryBroadcast + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- CometBroadcastExchange : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.item @@ -45,12 +44,11 @@ CometNativeColumnarToRow : : +- CometBroadcastHashJoin : : :- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.inventory - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometSubqueryBroadcast + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- CometBroadcastExchange : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.item @@ -62,4 +60,4 @@ CometNativeColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.date_dim -Comet accelerated 56 out of 60 eligible operators (93%). Final plan contains 3 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 58 out of 60 eligible operators (96%). Final plan contains 1 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q4.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q4.native_datafusion/extended.txt index b2914584b4..bf61cfd81e 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q4.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q4.native_datafusion/extended.txt @@ -23,11 +23,10 @@ CometNativeColumnarToRow : : : : : : +- CometBroadcastExchange : : : : : : +- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometNativeColumnarToRow - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : +- CometSubqueryBroadcast + : : : : : : +- CometBroadcastExchange + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : : +- CometBroadcastExchange : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -45,11 +44,10 @@ CometNativeColumnarToRow : : : : : +- CometBroadcastExchange : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- CometSubqueryBroadcast + : : : : : +- CometBroadcastExchange + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : +- CometBroadcastExchange : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -68,11 +66,10 @@ CometNativeColumnarToRow : : : : +- CometBroadcastExchange : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometSubqueryBroadcast + : : : : +- CometBroadcastExchange + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- CometBroadcastExchange : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -90,11 +87,10 @@ CometNativeColumnarToRow : : : +- CometBroadcastExchange : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometSubqueryBroadcast + : : : +- CometBroadcastExchange + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- CometBroadcastExchange : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -113,11 +109,10 @@ CometNativeColumnarToRow : : +- CometBroadcastExchange : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometSubqueryBroadcast + : : +- CometBroadcastExchange + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim : +- CometBroadcastExchange : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -135,13 +130,12 @@ CometNativeColumnarToRow : +- CometBroadcastExchange : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.web_sales - : +- SubqueryBroadcast - : +- BroadcastExchange - : +- CometNativeColumnarToRow - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometSubqueryBroadcast + : +- CometBroadcastExchange + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.date_dim +- CometBroadcastExchange +- CometFilter +- CometNativeScan parquet spark_catalog.default.date_dim -Comet accelerated 126 out of 138 eligible operators (91%). Final plan contains 7 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 132 out of 138 eligible operators (95%). Final plan contains 1 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q40.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q40.native_datafusion/extended.txt index 9b00a2c6a4..1a148d7ab4 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q40.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q40.native_datafusion/extended.txt @@ -15,11 +15,10 @@ CometNativeColumnarToRow : : : : +- CometExchange : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometSubqueryBroadcast + : : : : +- CometBroadcastExchange + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- CometSort : : : +- CometExchange : : : +- CometProject @@ -37,4 +36,4 @@ CometNativeColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.date_dim -Comet accelerated 34 out of 36 eligible operators (94%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 35 out of 36 eligible operators (97%). Final plan contains 1 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q45.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q45.native_datafusion/extended.txt index 7292a238d5..e7658f2548 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q45.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q45.native_datafusion/extended.txt @@ -17,12 +17,11 @@ TakeOrderedAndProject : : : : +- CometBroadcastHashJoin : : : : :- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- CometSubqueryBroadcast + : : : : : +- CometBroadcastExchange + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : +- CometBroadcastExchange : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.customer @@ -44,4 +43,4 @@ TakeOrderedAndProject +- CometFilter +- CometNativeScan parquet spark_catalog.default.item -Comet accelerated 32 out of 41 eligible operators (78%). Final plan contains 4 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 33 out of 41 eligible operators (80%). Final plan contains 3 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q46.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q46.native_datafusion/extended.txt index dbdaaa275a..f58213e7cb 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q46.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q46.native_datafusion/extended.txt @@ -17,12 +17,11 @@ CometNativeColumnarToRow : : : : : +- CometBroadcastHashJoin : : : : : :- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometNativeColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : +- CometSubqueryBroadcast + : : : : : : +- CometBroadcastExchange + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : : +- CometBroadcastExchange : : : : : +- CometProject : : : : : +- CometFilter @@ -46,4 +45,4 @@ CometNativeColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.customer_address -Comet accelerated 43 out of 45 eligible operators (95%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 44 out of 45 eligible operators (97%). Final plan contains 1 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q47.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q47.native_datafusion/extended.txt index 0886059452..54fa1daf2e 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q47.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q47.native_datafusion/extended.txt @@ -26,11 +26,10 @@ TakeOrderedAndProject : : : : +- CometBroadcastExchange : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometSubqueryBroadcast + : : : : +- CometBroadcastExchange + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- CometBroadcastExchange : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -58,11 +57,10 @@ TakeOrderedAndProject : : : +- CometBroadcastExchange : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometSubqueryBroadcast + : : : +- CometBroadcastExchange + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- CometBroadcastExchange : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -90,11 +88,10 @@ TakeOrderedAndProject : : +- CometBroadcastExchange : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometSubqueryBroadcast + : : +- CometBroadcastExchange + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim : +- CometBroadcastExchange : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -102,4 +99,4 @@ TakeOrderedAndProject +- CometFilter +- CometNativeScan parquet spark_catalog.default.store -Comet accelerated 75 out of 97 eligible operators (77%). Final plan contains 6 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 78 out of 97 eligible operators (80%). Final plan contains 3 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q48.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q48.native_datafusion/extended.txt index 75a1c725c2..4b1541c674 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q48.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q48.native_datafusion/extended.txt @@ -12,12 +12,11 @@ CometNativeColumnarToRow : : : +- CometBroadcastHashJoin : : : :- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometSubqueryBroadcast + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- CometBroadcastExchange : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.store @@ -34,4 +33,4 @@ CometNativeColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.date_dim -Comet accelerated 31 out of 33 eligible operators (93%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 32 out of 33 eligible operators (96%). Final plan contains 1 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q49.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q49.native_datafusion/extended.txt index 6d7dadcde9..f2ae44fb86 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q49.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q49.native_datafusion/extended.txt @@ -23,12 +23,11 @@ CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometSubqueryBroadcast + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- CometProject : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.web_returns @@ -55,12 +54,11 @@ CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometSubqueryBroadcast + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- CometProject : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.catalog_returns @@ -87,12 +85,11 @@ CometNativeColumnarToRow : : +- CometProject : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometSubqueryBroadcast + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim : +- CometProject : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.store_returns @@ -101,4 +98,4 @@ CometNativeColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.date_dim -Comet accelerated 72 out of 95 eligible operators (75%). Final plan contains 7 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 75 out of 95 eligible operators (78%). Final plan contains 4 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q5.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q5.native_datafusion/extended.txt index 719229a9a3..adef074558 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q5.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q5.native_datafusion/extended.txt @@ -16,21 +16,19 @@ CometNativeColumnarToRow : : : :- CometProject : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometSubqueryBroadcast + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.store_returns - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometSubqueryBroadcast + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- CometBroadcastExchange : : +- CometProject : : +- CometFilter @@ -50,21 +48,19 @@ CometNativeColumnarToRow : : : :- CometProject : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometSubqueryBroadcast + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.catalog_returns - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometSubqueryBroadcast + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- CometBroadcastExchange : : +- CometProject : : +- CometFilter @@ -84,22 +80,20 @@ CometNativeColumnarToRow : : :- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometSubqueryBroadcast + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- CometProject : : +- CometBroadcastHashJoin : : :- CometBroadcastExchange : : : +- CometNativeScan parquet spark_catalog.default.web_returns - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometSubqueryBroadcast + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- CometProject : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.web_sales @@ -112,4 +106,4 @@ CometNativeColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.web_site -Comet accelerated 94 out of 106 eligible operators (88%). Final plan contains 7 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 100 out of 106 eligible operators (94%). Final plan contains 1 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q50.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q50.native_datafusion/extended.txt index c013381de0..e20ab4a6d6 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q50.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q50.native_datafusion/extended.txt @@ -16,12 +16,11 @@ CometNativeColumnarToRow : : : +- CometBroadcastExchange : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.store_returns - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometSubqueryBroadcast + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- CometBroadcastExchange : : +- CometProject : : +- CometFilter @@ -34,4 +33,4 @@ CometNativeColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.date_dim -Comet accelerated 31 out of 33 eligible operators (93%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 32 out of 33 eligible operators (96%). Final plan contains 1 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q51.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q51.native_datafusion/extended.txt index 425e772118..173d1f064f 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q51.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q51.native_datafusion/extended.txt @@ -20,12 +20,11 @@ TakeOrderedAndProject : +- CometBroadcastHashJoin : :- CometFilter : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometSubqueryBroadcast + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim : +- CometBroadcastExchange : +- CometProject : +- CometFilter @@ -44,15 +43,14 @@ TakeOrderedAndProject +- CometBroadcastHashJoin :- CometFilter : +- CometNativeScan parquet spark_catalog.default.store_sales - : +- SubqueryBroadcast - : +- BroadcastExchange - : +- CometNativeColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometSubqueryBroadcast + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.date_dim +- CometBroadcastExchange +- CometProject +- CometFilter +- CometNativeScan parquet spark_catalog.default.date_dim -Comet accelerated 40 out of 51 eligible operators (78%). Final plan contains 5 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 42 out of 51 eligible operators (82%). Final plan contains 3 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q53.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q53.native_datafusion/extended.txt index 936cde052f..9a8a9bbba4 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q53.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q53.native_datafusion/extended.txt @@ -20,12 +20,11 @@ TakeOrderedAndProject : : +- CometBroadcastExchange : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometSubqueryBroadcast + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim : +- CometBroadcastExchange : +- CometProject : +- CometFilter @@ -34,4 +33,4 @@ TakeOrderedAndProject +- CometFilter +- CometNativeScan parquet spark_catalog.default.store -Comet accelerated 27 out of 33 eligible operators (81%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 28 out of 33 eligible operators (84%). Final plan contains 1 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q54.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q54.native_datafusion/extended.txt index e7fd40210f..9e1691c2fc 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q54.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q54.native_datafusion/extended.txt @@ -27,21 +27,19 @@ CometNativeColumnarToRow : : : : : : : :- CometProject : : : : : : : : +- CometFilter : : : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : : : : : +- SubqueryBroadcast - : : : : : : : : +- BroadcastExchange - : : : : : : : : +- CometNativeColumnarToRow - : : : : : : : : +- CometProject - : : : : : : : : +- CometFilter - : : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : : : +- CometSubqueryBroadcast + : : : : : : : : +- CometBroadcastExchange + : : : : : : : : +- CometProject + : : : : : : : : +- CometFilter + : : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : : : : +- CometProject : : : : : : : +- CometFilter : : : : : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : : : : : +- SubqueryBroadcast - : : : : : : : +- BroadcastExchange - : : : : : : : +- CometNativeColumnarToRow - : : : : : : : +- CometProject - : : : : : : : +- CometFilter - : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : : +- CometSubqueryBroadcast + : : : : : : : +- CometBroadcastExchange + : : : : : : : +- CometProject + : : : : : : : +- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : : : +- CometBroadcastExchange : : : : : : +- CometProject : : : : : : +- CometFilter @@ -56,28 +54,27 @@ CometNativeColumnarToRow : : : +- CometBroadcastExchange : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : : :- Subquery - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometHashAggregate - : : : : : +- CometExchange - : : : : : +- CometHashAggregate - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : +- Subquery - : : : : +- CometNativeColumnarToRow - : : : : +- CometHashAggregate - : : : : +- CometExchange - : : : : +- CometHashAggregate - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometSubqueryBroadcast + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : : :- Subquery + : : : : : +- CometNativeColumnarToRow + : : : : : +- CometHashAggregate + : : : : : +- CometExchange + : : : : : +- CometHashAggregate + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- Subquery + : : : : +- CometNativeColumnarToRow + : : : : +- CometHashAggregate + : : : : +- CometExchange + : : : : +- CometHashAggregate + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- CometBroadcastExchange : : +- CometProject : : +- CometFilter @@ -89,8 +86,22 @@ CometNativeColumnarToRow +- CometBroadcastExchange +- CometProject +- CometFilter - : :- ReusedSubquery - : +- ReusedSubquery + : :- Subquery + : : +- CometNativeColumnarToRow + : : +- CometHashAggregate + : : +- CometExchange + : : +- CometHashAggregate + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- Subquery + : +- CometNativeColumnarToRow + : +- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.date_dim +- CometNativeScan parquet spark_catalog.default.date_dim -Comet accelerated 78 out of 88 eligible operators (88%). Final plan contains 6 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 93 out of 100 eligible operators (93%). Final plan contains 5 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q56.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q56.native_datafusion/extended.txt index d52c40d57b..0744bc307d 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q56.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q56.native_datafusion/extended.txt @@ -15,12 +15,11 @@ CometNativeColumnarToRow : : : +- CometBroadcastHashJoin : : : :- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometSubqueryBroadcast + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter @@ -49,12 +48,11 @@ CometNativeColumnarToRow : : : +- CometBroadcastHashJoin : : : :- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometSubqueryBroadcast + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter @@ -83,12 +81,11 @@ CometNativeColumnarToRow : : +- CometBroadcastHashJoin : : :- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometSubqueryBroadcast + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- CometBroadcastExchange : : +- CometProject : : +- CometFilter @@ -107,4 +104,4 @@ CometNativeColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.item -Comet accelerated 98 out of 104 eligible operators (94%). Final plan contains 4 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 101 out of 104 eligible operators (97%). Final plan contains 1 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q57.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q57.native_datafusion/extended.txt index 024bfeeeea..aa4ccf3246 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q57.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q57.native_datafusion/extended.txt @@ -26,11 +26,10 @@ TakeOrderedAndProject : : : : +- CometBroadcastExchange : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometSubqueryBroadcast + : : : : +- CometBroadcastExchange + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- CometBroadcastExchange : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -58,11 +57,10 @@ TakeOrderedAndProject : : : +- CometBroadcastExchange : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometSubqueryBroadcast + : : : +- CometBroadcastExchange + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- CometBroadcastExchange : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -90,11 +88,10 @@ TakeOrderedAndProject : : +- CometBroadcastExchange : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometSubqueryBroadcast + : : +- CometBroadcastExchange + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim : +- CometBroadcastExchange : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -102,4 +99,4 @@ TakeOrderedAndProject +- CometFilter +- CometNativeScan parquet spark_catalog.default.call_center -Comet accelerated 75 out of 97 eligible operators (77%). Final plan contains 6 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 78 out of 97 eligible operators (80%). Final plan contains 3 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q58.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q58.native_datafusion/extended.txt index 6a2729349e..ed81d13b0e 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q58.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q58.native_datafusion/extended.txt @@ -14,22 +14,21 @@ CometNativeColumnarToRow : : : +- CometBroadcastHashJoin : : : :- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : : +- Subquery - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometSubqueryBroadcast + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : : +- Subquery + : : : : : +- CometNativeColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter @@ -59,22 +58,21 @@ CometNativeColumnarToRow : : +- CometBroadcastHashJoin : : :- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : : +- Subquery - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometSubqueryBroadcast + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : : +- Subquery + : : : : +- CometNativeColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- CometBroadcastExchange : : +- CometProject : : +- CometFilter @@ -100,22 +98,21 @@ CometNativeColumnarToRow : +- CometBroadcastHashJoin : :- CometFilter : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : : +- Subquery - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometSubqueryBroadcast + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : : +- Subquery + : : : +- CometNativeColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometNativeScan parquet spark_catalog.default.date_dim : +- CometBroadcastExchange : +- CometProject : +- CometFilter @@ -131,4 +128,4 @@ CometNativeColumnarToRow : +- ReusedSubquery +- CometNativeScan parquet spark_catalog.default.date_dim -Comet accelerated 112 out of 124 eligible operators (90%). Final plan contains 8 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 115 out of 124 eligible operators (92%). Final plan contains 5 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q6.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q6.native_datafusion/extended.txt index 1cb21cb650..4b31bf751a 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q6.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q6.native_datafusion/extended.txt @@ -21,24 +21,30 @@ CometNativeColumnarToRow : : +- CometBroadcastExchange : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometProject - : : +- CometFilter - : : : +- Subquery - : : : +- CometNativeColumnarToRow - : : : +- CometHashAggregate - : : : +- CometExchange - : : : +- CometHashAggregate - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometSubqueryBroadcast + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : : +- Subquery + : : : +- CometNativeColumnarToRow + : : : +- CometHashAggregate + : : : +- CometExchange + : : : +- CometHashAggregate + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometNativeScan parquet spark_catalog.default.date_dim : +- CometBroadcastExchange : +- CometProject : +- CometFilter - : : +- ReusedSubquery + : : +- Subquery + : : +- CometNativeColumnarToRow + : : +- CometHashAggregate + : : +- CometExchange + : : +- CometHashAggregate + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim : +- CometNativeScan parquet spark_catalog.default.date_dim +- CometBroadcastExchange +- CometProject @@ -54,4 +60,4 @@ CometNativeColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.item -Comet accelerated 48 out of 52 eligible operators (92%). Final plan contains 3 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 55 out of 58 eligible operators (94%). Final plan contains 3 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q60.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q60.native_datafusion/extended.txt index d52c40d57b..0744bc307d 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q60.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q60.native_datafusion/extended.txt @@ -15,12 +15,11 @@ CometNativeColumnarToRow : : : +- CometBroadcastHashJoin : : : :- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometSubqueryBroadcast + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter @@ -49,12 +48,11 @@ CometNativeColumnarToRow : : : +- CometBroadcastHashJoin : : : :- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometSubqueryBroadcast + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter @@ -83,12 +81,11 @@ CometNativeColumnarToRow : : +- CometBroadcastHashJoin : : :- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometSubqueryBroadcast + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- CometBroadcastExchange : : +- CometProject : : +- CometFilter @@ -107,4 +104,4 @@ CometNativeColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.item -Comet accelerated 98 out of 104 eligible operators (94%). Final plan contains 4 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 101 out of 104 eligible operators (97%). Final plan contains 1 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q61.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q61.native_datafusion/extended.txt index ef8907c7d8..8d91cbe7a2 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q61.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q61.native_datafusion/extended.txt @@ -18,12 +18,11 @@ Project : : : : : : +- CometBroadcastHashJoin : : : : : : :- CometFilter : : : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : : : : : +- SubqueryBroadcast - : : : : : : : +- BroadcastExchange - : : : : : : : +- CometNativeColumnarToRow - : : : : : : : +- CometProject - : : : : : : : +- CometFilter - : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : : +- CometSubqueryBroadcast + : : : : : : : +- CometBroadcastExchange + : : : : : : : +- CometProject + : : : : : : : +- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : : : +- CometBroadcastExchange : : : : : : +- CometProject : : : : : : +- CometFilter @@ -64,12 +63,11 @@ Project : : : : +- CometBroadcastHashJoin : : : : :- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- CometSubqueryBroadcast + : : : : : +- CometBroadcastExchange + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : +- CometBroadcastExchange : : : : +- CometProject : : : : +- CometFilter @@ -90,4 +88,4 @@ Project +- CometFilter +- CometNativeScan parquet spark_catalog.default.item -Comet accelerated 80 out of 87 eligible operators (91%). Final plan contains 4 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 82 out of 87 eligible operators (94%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q63.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q63.native_datafusion/extended.txt index 936cde052f..9a8a9bbba4 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q63.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q63.native_datafusion/extended.txt @@ -20,12 +20,11 @@ TakeOrderedAndProject : : +- CometBroadcastExchange : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometSubqueryBroadcast + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim : +- CometBroadcastExchange : +- CometProject : +- CometFilter @@ -34,4 +33,4 @@ TakeOrderedAndProject +- CometFilter +- CometNativeScan parquet spark_catalog.default.store -Comet accelerated 27 out of 33 eligible operators (81%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 28 out of 33 eligible operators (84%). Final plan contains 1 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q64.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q64.native_datafusion/extended.txt index f1fe6fb3e3..14ffe4eab9 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q64.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q64.native_datafusion/extended.txt @@ -46,11 +46,10 @@ CometNativeColumnarToRow : : : : : : : : : : : : : : : : : :- CometBroadcastExchange : : : : : : : : : : : : : : : : : : +- CometFilter : : : : : : : : : : : : : : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : : : : : : : : : : : : : : : : +- SubqueryBroadcast - : : : : : : : : : : : : : : : : : : +- BroadcastExchange - : : : : : : : : : : : : : : : : : : +- CometNativeColumnarToRow - : : : : : : : : : : : : : : : : : : +- CometFilter - : : : : : : : : : : : : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : : : : : : : : : : : : : +- CometSubqueryBroadcast + : : : : : : : : : : : : : : : : : : +- CometBroadcastExchange + : : : : : : : : : : : : : : : : : : +- CometFilter + : : : : : : : : : : : : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : : : : : : : : : : : : : : +- CometProject : : : : : : : : : : : : : : : : : +- CometFilter : : : : : : : : : : : : : : : : : +- CometNativeScan parquet spark_catalog.default.store_returns @@ -166,11 +165,10 @@ CometNativeColumnarToRow : : : : : : : : : : : : : : : : :- CometBroadcastExchange : : : : : : : : : : : : : : : : : +- CometFilter : : : : : : : : : : : : : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : : : : : : : : : : : : : : : +- SubqueryBroadcast - : : : : : : : : : : : : : : : : : +- BroadcastExchange - : : : : : : : : : : : : : : : : : +- CometNativeColumnarToRow - : : : : : : : : : : : : : : : : : +- CometFilter - : : : : : : : : : : : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : : : : : : : : : : : : +- CometSubqueryBroadcast + : : : : : : : : : : : : : : : : : +- CometBroadcastExchange + : : : : : : : : : : : : : : : : : +- CometFilter + : : : : : : : : : : : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : : : : : : : : : : : : : +- CometProject : : : : : : : : : : : : : : : : +- CometFilter : : : : : : : : : : : : : : : : +- CometNativeScan parquet spark_catalog.default.store_returns @@ -244,4 +242,4 @@ CometNativeColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.item -Comet accelerated 238 out of 242 eligible operators (98%). Final plan contains 3 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 240 out of 242 eligible operators (99%). Final plan contains 1 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q65.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q65.native_datafusion/extended.txt index 167a3386c2..21d60497a1 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q65.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q65.native_datafusion/extended.txt @@ -17,12 +17,11 @@ CometNativeColumnarToRow : : +- CometBroadcastHashJoin : : :- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometSubqueryBroadcast + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- CometBroadcastExchange : : +- CometProject : : +- CometFilter @@ -43,15 +42,14 @@ CometNativeColumnarToRow +- CometBroadcastHashJoin :- CometFilter : +- CometNativeScan parquet spark_catalog.default.store_sales - : +- SubqueryBroadcast - : +- BroadcastExchange - : +- CometNativeColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometSubqueryBroadcast + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.date_dim +- CometBroadcastExchange +- CometProject +- CometFilter +- CometNativeScan parquet spark_catalog.default.date_dim -Comet accelerated 48 out of 52 eligible operators (92%). Final plan contains 3 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 50 out of 52 eligible operators (96%). Final plan contains 1 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q66.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q66.native_datafusion/extended.txt index a7423e3a59..c07ecfe46a 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q66.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q66.native_datafusion/extended.txt @@ -17,11 +17,10 @@ CometNativeColumnarToRow : : : : +- CometBroadcastHashJoin : : : : :- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- CometSubqueryBroadcast + : : : : : +- CometBroadcastExchange + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : +- CometBroadcastExchange : : : : +- CometProject : : : : +- CometFilter @@ -50,11 +49,10 @@ CometNativeColumnarToRow : : : +- CometBroadcastHashJoin : : : :- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometSubqueryBroadcast + : : : : +- CometBroadcastExchange + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter @@ -71,4 +69,4 @@ CometNativeColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.ship_mode -Comet accelerated 65 out of 69 eligible operators (94%). Final plan contains 3 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 67 out of 69 eligible operators (97%). Final plan contains 1 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q67.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q67.native_datafusion/extended.txt index 892f37cf2d..00dc622681 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q67.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q67.native_datafusion/extended.txt @@ -16,12 +16,11 @@ TakeOrderedAndProject : : +- CometBroadcastHashJoin : : :- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometSubqueryBroadcast + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- CometBroadcastExchange : : +- CometProject : : +- CometFilter @@ -35,4 +34,4 @@ TakeOrderedAndProject +- CometFilter +- CometNativeScan parquet spark_catalog.default.item -Comet accelerated 29 out of 34 eligible operators (85%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 30 out of 34 eligible operators (88%). Final plan contains 1 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q68.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q68.native_datafusion/extended.txt index dbdaaa275a..f58213e7cb 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q68.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q68.native_datafusion/extended.txt @@ -17,12 +17,11 @@ CometNativeColumnarToRow : : : : : +- CometBroadcastHashJoin : : : : : :- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometNativeColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : +- CometSubqueryBroadcast + : : : : : : +- CometBroadcastExchange + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : : +- CometBroadcastExchange : : : : : +- CometProject : : : : : +- CometFilter @@ -46,4 +45,4 @@ CometNativeColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.customer_address -Comet accelerated 43 out of 45 eligible operators (95%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 44 out of 45 eligible operators (97%). Final plan contains 1 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q69.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q69.native_datafusion/extended.txt index 3e0ba48add..6d398580fb 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q69.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q69.native_datafusion/extended.txt @@ -18,12 +18,11 @@ TakeOrderedAndProject : : : : +- CometProject : : : : +- CometBroadcastHashJoin : : : : :- CometNativeScan parquet spark_catalog.default.store_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- CometSubqueryBroadcast + : : : : : +- CometBroadcastExchange + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : +- CometBroadcastExchange : : : : +- CometProject : : : : +- CometFilter @@ -33,12 +32,11 @@ TakeOrderedAndProject : : : +- CometProject : : : +- CometBroadcastHashJoin : : : :- CometNativeScan parquet spark_catalog.default.web_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometSubqueryBroadcast + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter @@ -48,12 +46,11 @@ TakeOrderedAndProject : : +- CometProject : : +- CometBroadcastHashJoin : : :- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometSubqueryBroadcast + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- CometBroadcastExchange : : +- CometProject : : +- CometFilter @@ -69,4 +66,4 @@ TakeOrderedAndProject +- CometFilter +- CometNativeScan parquet spark_catalog.default.customer_demographics -Comet accelerated 41 out of 61 eligible operators (67%). Final plan contains 9 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 44 out of 61 eligible operators (72%). Final plan contains 6 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q7.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q7.native_datafusion/extended.txt index 42206d1be5..8750081c55 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q7.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q7.native_datafusion/extended.txt @@ -13,12 +13,11 @@ CometNativeColumnarToRow : : : +- CometBroadcastHashJoin : : : :- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometSubqueryBroadcast + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter @@ -36,4 +35,4 @@ CometNativeColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.promotion -Comet accelerated 33 out of 35 eligible operators (94%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 34 out of 35 eligible operators (97%). Final plan contains 1 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q70.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q70.native_datafusion/extended.txt index 7c3d343454..7655338234 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q70.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q70.native_datafusion/extended.txt @@ -16,12 +16,11 @@ TakeOrderedAndProject : +- CometBroadcastHashJoin : :- CometFilter : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometSubqueryBroadcast + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim : +- CometBroadcastExchange : +- CometProject : +- CometFilter @@ -47,12 +46,11 @@ TakeOrderedAndProject : +- CometBroadcastHashJoin : :- CometFilter : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometSubqueryBroadcast + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim : +- CometBroadcastExchange : +- CometProject : +- CometFilter @@ -62,4 +60,4 @@ TakeOrderedAndProject +- CometFilter +- CometNativeScan parquet spark_catalog.default.date_dim -Comet accelerated 37 out of 56 eligible operators (66%). Final plan contains 7 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 39 out of 56 eligible operators (69%). Final plan contains 5 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q71.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q71.native_datafusion/extended.txt index 7f76dd192d..834098d4ee 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q71.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q71.native_datafusion/extended.txt @@ -17,12 +17,11 @@ CometNativeColumnarToRow : : +- CometBroadcastHashJoin : : :- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometSubqueryBroadcast + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- CometBroadcastExchange : : +- CometProject : : +- CometFilter @@ -31,12 +30,11 @@ CometNativeColumnarToRow : : +- CometBroadcastHashJoin : : :- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometSubqueryBroadcast + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- CometBroadcastExchange : : +- CometProject : : +- CometFilter @@ -45,12 +43,11 @@ CometNativeColumnarToRow : +- CometBroadcastHashJoin : :- CometFilter : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometSubqueryBroadcast + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim : +- CometBroadcastExchange : +- CometProject : +- CometFilter @@ -60,4 +57,4 @@ CometNativeColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.time_dim -Comet accelerated 51 out of 57 eligible operators (89%). Final plan contains 4 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 54 out of 57 eligible operators (94%). Final plan contains 1 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q72.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q72.native_datafusion/extended.txt index 623a68ddab..d443b31d1e 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q72.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q72.native_datafusion/extended.txt @@ -27,12 +27,11 @@ CometNativeColumnarToRow : : : : : : : : : +- CometBroadcastHashJoin : : : : : : : : : :- CometFilter : : : : : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : : : : : : : +- SubqueryBroadcast - : : : : : : : : : : +- BroadcastExchange - : : : : : : : : : : +- CometNativeColumnarToRow - : : : : : : : : : : +- CometProject - : : : : : : : : : : +- CometFilter - : : : : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : : : : : +- CometSubqueryBroadcast + : : : : : : : : : : +- CometBroadcastExchange + : : : : : : : : : : +- CometProject + : : : : : : : : : : +- CometFilter + : : : : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : : : : : : +- CometBroadcastExchange : : : : : : : : : +- CometFilter : : : : : : : : : +- CometNativeScan parquet spark_catalog.default.inventory @@ -69,4 +68,4 @@ CometNativeColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.catalog_returns -Comet accelerated 66 out of 68 eligible operators (97%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 67 out of 68 eligible operators (98%). Final plan contains 1 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q73.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q73.native_datafusion/extended.txt index 1b61be2937..4c0ef9abcd 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q73.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q73.native_datafusion/extended.txt @@ -15,12 +15,11 @@ CometNativeColumnarToRow : : : +- CometBroadcastHashJoin : : : :- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometSubqueryBroadcast + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter @@ -38,4 +37,4 @@ CometNativeColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.customer -Comet accelerated 35 out of 37 eligible operators (94%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 36 out of 37 eligible operators (97%). Final plan contains 1 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q74.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q74.native_datafusion/extended.txt index 51a3f5ca94..7768138645 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q74.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q74.native_datafusion/extended.txt @@ -19,11 +19,10 @@ CometNativeColumnarToRow : : : : +- CometBroadcastExchange : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometSubqueryBroadcast + : : : : +- CometBroadcastExchange + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- CometBroadcastExchange : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -41,11 +40,10 @@ CometNativeColumnarToRow : : : +- CometBroadcastExchange : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometSubqueryBroadcast + : : : +- CometBroadcastExchange + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- CometBroadcastExchange : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -64,11 +62,10 @@ CometNativeColumnarToRow : : +- CometBroadcastExchange : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometSubqueryBroadcast + : : +- CometBroadcastExchange + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim : +- CometBroadcastExchange : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -86,13 +83,12 @@ CometNativeColumnarToRow : +- CometBroadcastExchange : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.web_sales - : +- SubqueryBroadcast - : +- BroadcastExchange - : +- CometNativeColumnarToRow - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometSubqueryBroadcast + : +- CometBroadcastExchange + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.date_dim +- CometBroadcastExchange +- CometFilter +- CometNativeScan parquet spark_catalog.default.date_dim -Comet accelerated 83 out of 91 eligible operators (91%). Final plan contains 5 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 87 out of 91 eligible operators (95%). Final plan contains 1 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q75.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q75.native_datafusion/extended.txt index a943308f1e..ee7c0af216 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q75.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q75.native_datafusion/extended.txt @@ -22,11 +22,10 @@ CometNativeColumnarToRow : : : : +- CometBroadcastHashJoin : : : : :- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- CometSubqueryBroadcast + : : : : : +- CometBroadcastExchange + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : +- CometBroadcastExchange : : : : +- CometProject : : : : +- CometFilter @@ -49,11 +48,10 @@ CometNativeColumnarToRow : : : : +- CometBroadcastHashJoin : : : : :- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- CometSubqueryBroadcast + : : : : : +- CometBroadcastExchange + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : +- CometBroadcastExchange : : : : +- CometProject : : : : +- CometFilter @@ -76,11 +74,10 @@ CometNativeColumnarToRow : : : +- CometBroadcastHashJoin : : : :- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometSubqueryBroadcast + : : : : +- CometBroadcastExchange + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter @@ -113,11 +110,10 @@ CometNativeColumnarToRow : : : +- CometBroadcastHashJoin : : : :- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometSubqueryBroadcast + : : : : +- CometBroadcastExchange + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter @@ -140,11 +136,10 @@ CometNativeColumnarToRow : : : +- CometBroadcastHashJoin : : : :- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometSubqueryBroadcast + : : : : +- CometBroadcastExchange + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter @@ -167,11 +162,10 @@ CometNativeColumnarToRow : : +- CometBroadcastHashJoin : : :- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometSubqueryBroadcast + : : : +- CometBroadcastExchange + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- CometBroadcastExchange : : +- CometProject : : +- CometFilter @@ -185,4 +179,4 @@ CometNativeColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.web_returns -Comet accelerated 167 out of 179 eligible operators (93%). Final plan contains 7 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 173 out of 179 eligible operators (96%). Final plan contains 1 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q77.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q77.native_datafusion/extended.txt index 835f7a449a..8805a2e6c2 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q77.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q77.native_datafusion/extended.txt @@ -17,12 +17,11 @@ TakeOrderedAndProject : : : +- CometBroadcastHashJoin : : : :- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometSubqueryBroadcast + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter @@ -40,12 +39,11 @@ TakeOrderedAndProject : : +- CometBroadcastHashJoin : : :- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.store_returns - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometSubqueryBroadcast + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- CometBroadcastExchange : : +- CometProject : : +- CometFilter @@ -63,12 +61,11 @@ TakeOrderedAndProject : : +- CometProject : : +- CometBroadcastHashJoin : : :- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometSubqueryBroadcast + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- CometBroadcastExchange : : +- CometProject : : +- CometFilter @@ -80,12 +77,11 @@ TakeOrderedAndProject : +- CometProject : +- CometBroadcastHashJoin : :- CometNativeScan parquet spark_catalog.default.catalog_returns - : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometSubqueryBroadcast + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim : +- CometBroadcastExchange : +- CometProject : +- CometFilter @@ -102,12 +98,11 @@ TakeOrderedAndProject : : +- CometBroadcastHashJoin : : :- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometSubqueryBroadcast + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- CometBroadcastExchange : : +- CometProject : : +- CometFilter @@ -125,12 +120,11 @@ TakeOrderedAndProject : +- CometBroadcastHashJoin : :- CometFilter : : +- CometNativeScan parquet spark_catalog.default.web_returns - : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometSubqueryBroadcast + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim : +- CometBroadcastExchange : +- CometProject : +- CometFilter @@ -139,4 +133,4 @@ TakeOrderedAndProject +- CometFilter +- CometNativeScan parquet spark_catalog.default.web_page -Comet accelerated 109 out of 129 eligible operators (84%). Final plan contains 11 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 115 out of 129 eligible operators (89%). Final plan contains 5 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q78.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q78.native_datafusion/extended.txt index ebfdcc3c62..8426b9653c 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q78.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q78.native_datafusion/extended.txt @@ -17,11 +17,10 @@ TakeOrderedAndProject : : : : +- CometExchange : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometSubqueryBroadcast + : : : : +- CometBroadcastExchange + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- CometSort : : : +- CometExchange : : : +- CometProject @@ -44,11 +43,10 @@ TakeOrderedAndProject : : : +- CometExchange : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometSubqueryBroadcast + : : : +- CometBroadcastExchange + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- CometSort : : +- CometExchange : : +- CometProject @@ -71,11 +69,10 @@ TakeOrderedAndProject : : +- CometExchange : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometSubqueryBroadcast + : : +- CometBroadcastExchange + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim : +- CometSort : +- CometExchange : +- CometProject @@ -85,4 +82,4 @@ TakeOrderedAndProject +- CometFilter +- CometNativeScan parquet spark_catalog.default.date_dim -Comet accelerated 74 out of 82 eligible operators (90%). Final plan contains 4 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 77 out of 82 eligible operators (93%). Final plan contains 1 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q79.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q79.native_datafusion/extended.txt index 526b86f475..5af6449428 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q79.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q79.native_datafusion/extended.txt @@ -13,12 +13,11 @@ CometNativeColumnarToRow : : : +- CometBroadcastHashJoin : : : :- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometSubqueryBroadcast + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter @@ -36,4 +35,4 @@ CometNativeColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.customer -Comet accelerated 33 out of 35 eligible operators (94%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 34 out of 35 eligible operators (97%). Final plan contains 1 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q8.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q8.native_datafusion/extended.txt index cb04fc12c8..209a4e2bc0 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q8.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q8.native_datafusion/extended.txt @@ -11,12 +11,11 @@ CometNativeColumnarToRow : : +- CometBroadcastHashJoin : : :- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometSubqueryBroadcast + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- CometBroadcastExchange : : +- CometProject : : +- CometFilter @@ -49,4 +48,4 @@ CometNativeColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.customer -Comet accelerated 46 out of 48 eligible operators (95%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 47 out of 48 eligible operators (97%). Final plan contains 1 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q80.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q80.native_datafusion/extended.txt index 003be45ee5..27e32a1602 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q80.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q80.native_datafusion/extended.txt @@ -22,12 +22,11 @@ CometNativeColumnarToRow : : : : : : +- CometExchange : : : : : : +- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometNativeColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : +- CometSubqueryBroadcast + : : : : : : +- CometBroadcastExchange + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : : +- CometSort : : : : : +- CometExchange : : : : : +- CometProject @@ -66,12 +65,11 @@ CometNativeColumnarToRow : : : : : : +- CometExchange : : : : : : +- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometNativeColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : +- CometSubqueryBroadcast + : : : : : : +- CometBroadcastExchange + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : : +- CometSort : : : : : +- CometExchange : : : : : +- CometProject @@ -110,12 +108,11 @@ CometNativeColumnarToRow : : : : : +- CometExchange : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- CometSubqueryBroadcast + : : : : : +- CometBroadcastExchange + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : +- CometSort : : : : +- CometExchange : : : : +- CometProject @@ -138,4 +135,4 @@ CometNativeColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.promotion -Comet accelerated 129 out of 135 eligible operators (95%). Final plan contains 4 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 132 out of 135 eligible operators (97%). Final plan contains 1 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q81.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q81.native_datafusion/extended.txt index 949d50a867..6d9aa9697d 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q81.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q81.native_datafusion/extended.txt @@ -16,12 +16,11 @@ CometNativeColumnarToRow : : : : +- CometBroadcastHashJoin : : : : :- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_returns - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- CometSubqueryBroadcast + : : : : : +- CometBroadcastExchange + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : +- CometBroadcastExchange : : : : +- CometProject : : : : +- CometFilter @@ -44,12 +43,11 @@ CometNativeColumnarToRow : : : +- CometBroadcastHashJoin : : : :- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.catalog_returns - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometSubqueryBroadcast + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter @@ -67,4 +65,4 @@ CometNativeColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.customer_address -Comet accelerated 61 out of 65 eligible operators (93%). Final plan contains 3 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 63 out of 65 eligible operators (96%). Final plan contains 1 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q82.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q82.native_datafusion/extended.txt index 30fca82ec8..e7b75e3a55 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q82.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q82.native_datafusion/extended.txt @@ -17,12 +17,11 @@ CometNativeColumnarToRow : : +- CometProject : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.inventory - : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometSubqueryBroadcast + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim : +- CometBroadcastExchange : +- CometProject : +- CometFilter @@ -31,4 +30,4 @@ CometNativeColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.store_sales -Comet accelerated 28 out of 30 eligible operators (93%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 29 out of 30 eligible operators (96%). Final plan contains 1 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q83.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q83.native_datafusion/extended.txt index 3a16430014..e4d1992492 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q83.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q83.native_datafusion/extended.txt @@ -13,21 +13,20 @@ CometNativeColumnarToRow : : : +- CometBroadcastHashJoin : : : :- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.store_returns - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometNativeScan parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometSubqueryBroadcast + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter @@ -55,21 +54,20 @@ CometNativeColumnarToRow : : +- CometBroadcastHashJoin : : :- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.catalog_returns - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometSubqueryBroadcast + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- CometBroadcastExchange : : +- CometProject : : +- CometFilter @@ -97,21 +95,20 @@ CometNativeColumnarToRow : +- CometBroadcastHashJoin : :- CometFilter : : +- CometNativeScan parquet spark_catalog.default.web_returns - : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometNativeScan parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometSubqueryBroadcast + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim : +- CometBroadcastExchange : +- CometProject : +- CometFilter @@ -130,4 +127,4 @@ CometNativeColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.date_dim -Comet accelerated 121 out of 127 eligible operators (95%). Final plan contains 4 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 124 out of 127 eligible operators (97%). Final plan contains 1 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q85.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q85.native_datafusion/extended.txt index 06537a5eec..13506e196f 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q85.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q85.native_datafusion/extended.txt @@ -20,12 +20,11 @@ CometNativeColumnarToRow : : : : : : :- CometBroadcastExchange : : : : : : : +- CometFilter : : : : : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : : : : : +- SubqueryBroadcast - : : : : : : : +- BroadcastExchange - : : : : : : : +- CometNativeColumnarToRow - : : : : : : : +- CometProject - : : : : : : : +- CometFilter - : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : : +- CometSubqueryBroadcast + : : : : : : : +- CometBroadcastExchange + : : : : : : : +- CometProject + : : : : : : : +- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : : : +- CometProject : : : : : : +- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.web_returns @@ -53,4 +52,4 @@ CometNativeColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.reason -Comet accelerated 50 out of 52 eligible operators (96%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 51 out of 52 eligible operators (98%). Final plan contains 1 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q86.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q86.native_datafusion/extended.txt index c9117e2ae0..bddf80b4d0 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q86.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q86.native_datafusion/extended.txt @@ -14,12 +14,11 @@ TakeOrderedAndProject : +- CometBroadcastHashJoin : :- CometFilter : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometSubqueryBroadcast + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim : +- CometBroadcastExchange : +- CometProject : +- CometFilter @@ -29,4 +28,4 @@ TakeOrderedAndProject +- CometFilter +- CometNativeScan parquet spark_catalog.default.item -Comet accelerated 23 out of 28 eligible operators (82%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 24 out of 28 eligible operators (85%). Final plan contains 1 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q87.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q87.native_datafusion/extended.txt index 6676ec7a4b..5c80165c1f 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q87.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q87.native_datafusion/extended.txt @@ -15,12 +15,11 @@ HashAggregate : : : +- CometBroadcastHashJoin : : : :- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometSubqueryBroadcast + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter @@ -40,12 +39,11 @@ HashAggregate : : +- CometBroadcastHashJoin : : :- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometSubqueryBroadcast + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- CometBroadcastExchange : : +- CometProject : : +- CometFilter @@ -65,12 +63,11 @@ HashAggregate : +- CometBroadcastHashJoin : :- CometFilter : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometSubqueryBroadcast + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim : +- CometBroadcastExchange : +- CometProject : +- CometFilter @@ -80,4 +77,4 @@ HashAggregate +- CometFilter +- CometNativeScan parquet spark_catalog.default.customer -Comet accelerated 61 out of 74 eligible operators (82%). Final plan contains 7 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 64 out of 74 eligible operators (86%). Final plan contains 4 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q89.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q89.native_datafusion/extended.txt index 936cde052f..9a8a9bbba4 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q89.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q89.native_datafusion/extended.txt @@ -20,12 +20,11 @@ TakeOrderedAndProject : : +- CometBroadcastExchange : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometSubqueryBroadcast + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim : +- CometBroadcastExchange : +- CometProject : +- CometFilter @@ -34,4 +33,4 @@ TakeOrderedAndProject +- CometFilter +- CometNativeScan parquet spark_catalog.default.store -Comet accelerated 27 out of 33 eligible operators (81%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 28 out of 33 eligible operators (84%). Final plan contains 1 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q91.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q91.native_datafusion/extended.txt index 28bc6eab2f..9b191f8492 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q91.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q91.native_datafusion/extended.txt @@ -22,12 +22,11 @@ CometNativeColumnarToRow : : : : : +- CometBroadcastExchange : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_returns - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- CometSubqueryBroadcast + : : : : : +- CometBroadcastExchange + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : +- CometBroadcastExchange : : : : +- CometProject : : : : +- CometFilter @@ -48,4 +47,4 @@ CometNativeColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.household_demographics -Comet accelerated 45 out of 47 eligible operators (95%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 46 out of 47 eligible operators (97%). Final plan contains 1 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q92.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q92.native_datafusion/extended.txt index 6c5b57b3a1..40926fafac 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q92.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q92.native_datafusion/extended.txt @@ -10,12 +10,11 @@ CometNativeColumnarToRow : : +- CometBroadcastHashJoin : : :- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometSubqueryBroadcast + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- CometBroadcastExchange : : +- CometProject : : +- CometFilter @@ -29,12 +28,11 @@ CometNativeColumnarToRow : +- CometBroadcastHashJoin : :- CometFilter : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometSubqueryBroadcast + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim : +- CometBroadcastExchange : +- CometProject : +- CometFilter @@ -44,4 +42,4 @@ CometNativeColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.date_dim -Comet accelerated 38 out of 42 eligible operators (90%). Final plan contains 3 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 40 out of 42 eligible operators (95%). Final plan contains 1 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q97.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q97.native_datafusion/extended.txt index b63930a880..74163dff2b 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q97.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q97.native_datafusion/extended.txt @@ -11,12 +11,11 @@ CometNativeColumnarToRow : +- CometProject : +- CometBroadcastHashJoin : :- CometNativeScan parquet spark_catalog.default.store_sales - : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometSubqueryBroadcast + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim : +- CometBroadcastExchange : +- CometProject : +- CometFilter @@ -28,15 +27,14 @@ CometNativeColumnarToRow +- CometProject +- CometBroadcastHashJoin :- CometNativeScan parquet spark_catalog.default.catalog_sales - : +- SubqueryBroadcast - : +- BroadcastExchange - : +- CometNativeColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometSubqueryBroadcast + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.date_dim +- CometBroadcastExchange +- CometProject +- CometFilter +- CometNativeScan parquet spark_catalog.default.date_dim -Comet accelerated 33 out of 37 eligible operators (89%). Final plan contains 3 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 35 out of 37 eligible operators (94%). Final plan contains 1 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q98.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q98.native_datafusion/extended.txt index 6cc173560b..f1fc5ff9a3 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q98.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q98.native_datafusion/extended.txt @@ -16,12 +16,11 @@ CometNativeColumnarToRow : +- CometBroadcastHashJoin : :- CometFilter : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometSubqueryBroadcast + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim : +- CometBroadcastExchange : +- CometProject : +- CometFilter @@ -31,4 +30,4 @@ CometNativeColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.date_dim -Comet accelerated 25 out of 29 eligible operators (86%). Final plan contains 3 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 26 out of 29 eligible operators (89%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q10a.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q10a.native_datafusion/extended.txt index cba125f412..173c450c71 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q10a.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q10a.native_datafusion/extended.txt @@ -16,12 +16,11 @@ CometNativeColumnarToRow : : : +- CometProject : : : +- CometBroadcastHashJoin : : : :- CometNativeScan parquet spark_catalog.default.store_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometSubqueryBroadcast + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter @@ -31,12 +30,11 @@ CometNativeColumnarToRow : : :- CometProject : : : +- CometBroadcastHashJoin : : : :- CometNativeScan parquet spark_catalog.default.web_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometSubqueryBroadcast + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter @@ -44,12 +42,11 @@ CometNativeColumnarToRow : : +- CometProject : : +- CometBroadcastHashJoin : : :- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometSubqueryBroadcast + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- CometBroadcastExchange : : +- CometProject : : +- CometFilter @@ -63,4 +60,4 @@ CometNativeColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.customer_demographics -Comet accelerated 54 out of 60 eligible operators (90%). Final plan contains 4 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 57 out of 60 eligible operators (95%). Final plan contains 1 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q11.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q11.native_datafusion/extended.txt index 51a3f5ca94..7768138645 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q11.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q11.native_datafusion/extended.txt @@ -19,11 +19,10 @@ CometNativeColumnarToRow : : : : +- CometBroadcastExchange : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometSubqueryBroadcast + : : : : +- CometBroadcastExchange + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- CometBroadcastExchange : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -41,11 +40,10 @@ CometNativeColumnarToRow : : : +- CometBroadcastExchange : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometSubqueryBroadcast + : : : +- CometBroadcastExchange + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- CometBroadcastExchange : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -64,11 +62,10 @@ CometNativeColumnarToRow : : +- CometBroadcastExchange : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometSubqueryBroadcast + : : +- CometBroadcastExchange + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim : +- CometBroadcastExchange : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -86,13 +83,12 @@ CometNativeColumnarToRow : +- CometBroadcastExchange : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.web_sales - : +- SubqueryBroadcast - : +- BroadcastExchange - : +- CometNativeColumnarToRow - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometSubqueryBroadcast + : +- CometBroadcastExchange + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.date_dim +- CometBroadcastExchange +- CometFilter +- CometNativeScan parquet spark_catalog.default.date_dim -Comet accelerated 83 out of 91 eligible operators (91%). Final plan contains 5 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 87 out of 91 eligible operators (95%). Final plan contains 1 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q12.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q12.native_datafusion/extended.txt index a8c2531a38..a6cc526735 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q12.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q12.native_datafusion/extended.txt @@ -13,12 +13,11 @@ TakeOrderedAndProject : +- CometBroadcastHashJoin : :- CometFilter : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometSubqueryBroadcast + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim : +- CometBroadcastExchange : +- CometProject : +- CometFilter @@ -28,4 +27,4 @@ TakeOrderedAndProject +- CometFilter +- CometNativeScan parquet spark_catalog.default.date_dim -Comet accelerated 22 out of 27 eligible operators (81%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 23 out of 27 eligible operators (85%). Final plan contains 1 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q14.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q14.native_datafusion/extended.txt index 50139381f2..2d20c40e4b 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q14.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q14.native_datafusion/extended.txt @@ -11,12 +11,11 @@ CometNativeColumnarToRow : : :- CometProject : : : +- CometBroadcastHashJoin : : : :- CometNativeScan parquet spark_catalog.default.store_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometSubqueryBroadcast + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter @@ -24,12 +23,11 @@ CometNativeColumnarToRow : : :- CometProject : : : +- CometBroadcastHashJoin : : : :- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometSubqueryBroadcast + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter @@ -37,12 +35,11 @@ CometNativeColumnarToRow : : +- CometProject : : +- CometBroadcastHashJoin : : :- CometNativeScan parquet spark_catalog.default.web_sales - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometSubqueryBroadcast + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- CometBroadcastExchange : : +- CometProject : : +- CometFilter @@ -57,17 +54,16 @@ CometNativeColumnarToRow : : :- CometBroadcastHashJoin : : : :- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : : +- Subquery - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometSubqueryBroadcast + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : : +- Subquery + : : : : : +- CometNativeColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometBroadcastHashJoin @@ -84,12 +80,11 @@ CometNativeColumnarToRow : : : : : +- CometBroadcastHashJoin : : : : : :- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometNativeColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : +- CometSubqueryBroadcast + : : : : : : +- CometBroadcastExchange + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : : +- CometBroadcastExchange : : : : : +- CometBroadcastHashJoin : : : : : :- CometFilter @@ -101,12 +96,11 @@ CometNativeColumnarToRow : : : : : : +- CometBroadcastHashJoin : : : : : : :- CometFilter : : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : : : : +- SubqueryBroadcast - : : : : : : : +- BroadcastExchange - : : : : : : : +- CometNativeColumnarToRow - : : : : : : : +- CometProject - : : : : : : : +- CometFilter - : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : : +- CometSubqueryBroadcast + : : : : : : : +- CometBroadcastExchange + : : : : : : : +- CometProject + : : : : : : : +- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : : : +- CometBroadcastExchange : : : : : : +- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.item @@ -125,12 +119,11 @@ CometNativeColumnarToRow : : : : +- CometBroadcastHashJoin : : : : :- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- CometSubqueryBroadcast + : : : : : +- CometBroadcastExchange + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : +- CometBroadcastExchange : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.item @@ -158,12 +151,11 @@ CometNativeColumnarToRow : : : : +- CometBroadcastHashJoin : : : : :- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- CometSubqueryBroadcast + : : : : : +- CometBroadcastExchange + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : +- CometBroadcastExchange : : : : +- CometBroadcastHashJoin : : : : :- CometFilter @@ -175,12 +167,11 @@ CometNativeColumnarToRow : : : : : +- CometBroadcastHashJoin : : : : : :- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometNativeColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : +- CometSubqueryBroadcast + : : : : : : +- CometBroadcastExchange + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : : +- CometBroadcastExchange : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.item @@ -199,12 +190,11 @@ CometNativeColumnarToRow : : : +- CometBroadcastHashJoin : : : :- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometSubqueryBroadcast + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- CometBroadcastExchange : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.item @@ -215,7 +205,11 @@ CometNativeColumnarToRow : +- CometBroadcastExchange : +- CometProject : +- CometFilter - : : +- ReusedSubquery + : : +- Subquery + : : +- CometNativeColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim : +- CometNativeScan parquet spark_catalog.default.date_dim +- CometBroadcastExchange +- CometFilter @@ -230,17 +224,16 @@ CometNativeColumnarToRow : :- CometBroadcastHashJoin : : :- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : : +- Subquery - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometSubqueryBroadcast + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : : +- Subquery + : : : : +- CometNativeColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- CometBroadcastExchange : : +- CometProject : : +- CometBroadcastHashJoin @@ -257,12 +250,11 @@ CometNativeColumnarToRow : : : : +- CometBroadcastHashJoin : : : : :- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- CometSubqueryBroadcast + : : : : : +- CometBroadcastExchange + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : +- CometBroadcastExchange : : : : +- CometBroadcastHashJoin : : : : :- CometFilter @@ -274,12 +266,11 @@ CometNativeColumnarToRow : : : : : +- CometBroadcastHashJoin : : : : : :- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometNativeColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : +- CometSubqueryBroadcast + : : : : : : +- CometBroadcastExchange + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : : +- CometBroadcastExchange : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.item @@ -298,12 +289,11 @@ CometNativeColumnarToRow : : : +- CometBroadcastHashJoin : : : :- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometSubqueryBroadcast + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- CometBroadcastExchange : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.item @@ -331,12 +321,11 @@ CometNativeColumnarToRow : : : +- CometBroadcastHashJoin : : : :- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometSubqueryBroadcast + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- CometBroadcastExchange : : : +- CometBroadcastHashJoin : : : :- CometFilter @@ -348,12 +337,11 @@ CometNativeColumnarToRow : : : : +- CometBroadcastHashJoin : : : : :- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- CometSubqueryBroadcast + : : : : : +- CometBroadcastExchange + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : +- CometBroadcastExchange : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.item @@ -372,12 +360,11 @@ CometNativeColumnarToRow : : +- CometBroadcastHashJoin : : :- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometSubqueryBroadcast + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- CometBroadcastExchange : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.item @@ -388,7 +375,11 @@ CometNativeColumnarToRow +- CometBroadcastExchange +- CometProject +- CometFilter - : +- ReusedSubquery + : +- Subquery + : +- CometNativeColumnarToRow + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.date_dim +- CometNativeScan parquet spark_catalog.default.date_dim -Comet accelerated 331 out of 371 eligible operators (89%). Final plan contains 21 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 354 out of 377 eligible operators (93%). Final plan contains 6 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q14a.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q14a.native_datafusion/extended.txt index 636e5ce377..83b545c34c 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q14a.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q14a.native_datafusion/extended.txt @@ -18,12 +18,11 @@ CometNativeColumnarToRow : : : :- CometProject : : : : +- CometBroadcastHashJoin : : : : :- CometNativeScan parquet spark_catalog.default.store_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- CometSubqueryBroadcast + : : : : : +- CometBroadcastExchange + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : +- CometBroadcastExchange : : : : +- CometProject : : : : +- CometFilter @@ -31,12 +30,11 @@ CometNativeColumnarToRow : : : :- CometProject : : : : +- CometBroadcastHashJoin : : : : :- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- CometSubqueryBroadcast + : : : : : +- CometBroadcastExchange + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : +- CometBroadcastExchange : : : : +- CometProject : : : : +- CometFilter @@ -44,12 +42,11 @@ CometNativeColumnarToRow : : : +- CometProject : : : +- CometBroadcastHashJoin : : : :- CometNativeScan parquet spark_catalog.default.web_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometSubqueryBroadcast + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter @@ -64,12 +61,11 @@ CometNativeColumnarToRow : : : :- CometBroadcastHashJoin : : : : :- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- CometSubqueryBroadcast + : : : : : +- CometBroadcastExchange + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : +- CometBroadcastExchange : : : : +- CometProject : : : : +- CometBroadcastHashJoin @@ -86,12 +82,11 @@ CometNativeColumnarToRow : : : : : : +- CometBroadcastHashJoin : : : : : : :- CometFilter : : : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : : : : : +- SubqueryBroadcast - : : : : : : : +- BroadcastExchange - : : : : : : : +- CometNativeColumnarToRow - : : : : : : : +- CometProject - : : : : : : : +- CometFilter - : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : : +- CometSubqueryBroadcast + : : : : : : : +- CometBroadcastExchange + : : : : : : : +- CometProject + : : : : : : : +- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : : : +- CometBroadcastExchange : : : : : : +- CometBroadcastHashJoin : : : : : : :- CometFilter @@ -103,12 +98,11 @@ CometNativeColumnarToRow : : : : : : : +- CometBroadcastHashJoin : : : : : : : :- CometFilter : : : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : : : : : +- SubqueryBroadcast - : : : : : : : : +- BroadcastExchange - : : : : : : : : +- CometNativeColumnarToRow - : : : : : : : : +- CometProject - : : : : : : : : +- CometFilter - : : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : : : +- CometSubqueryBroadcast + : : : : : : : : +- CometBroadcastExchange + : : : : : : : : +- CometProject + : : : : : : : : +- CometFilter + : : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : : : : +- CometBroadcastExchange : : : : : : : +- CometFilter : : : : : : : +- CometNativeScan parquet spark_catalog.default.item @@ -127,12 +121,11 @@ CometNativeColumnarToRow : : : : : +- CometBroadcastHashJoin : : : : : :- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometNativeColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : +- CometSubqueryBroadcast + : : : : : : +- CometBroadcastExchange + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : : +- CometBroadcastExchange : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.item @@ -160,12 +153,11 @@ CometNativeColumnarToRow : : : : : +- CometBroadcastHashJoin : : : : : :- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometNativeColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : +- CometSubqueryBroadcast + : : : : : : +- CometBroadcastExchange + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : : +- CometBroadcastExchange : : : : : +- CometBroadcastHashJoin : : : : : :- CometFilter @@ -177,12 +169,11 @@ CometNativeColumnarToRow : : : : : : +- CometBroadcastHashJoin : : : : : : :- CometFilter : : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : : : : +- SubqueryBroadcast - : : : : : : : +- BroadcastExchange - : : : : : : : +- CometNativeColumnarToRow - : : : : : : : +- CometProject - : : : : : : : +- CometFilter - : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : : +- CometSubqueryBroadcast + : : : : : : : +- CometBroadcastExchange + : : : : : : : +- CometProject + : : : : : : : +- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : : : +- CometBroadcastExchange : : : : : : +- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.item @@ -201,12 +192,11 @@ CometNativeColumnarToRow : : : : +- CometBroadcastHashJoin : : : : :- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- CometSubqueryBroadcast + : : : : : +- CometBroadcastExchange + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : +- CometBroadcastExchange : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.item @@ -230,12 +220,11 @@ CometNativeColumnarToRow : : : :- CometBroadcastHashJoin : : : : :- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- CometSubqueryBroadcast + : : : : : +- CometBroadcastExchange + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : +- CometBroadcastExchange : : : : +- CometProject : : : : +- CometBroadcastHashJoin @@ -252,12 +241,11 @@ CometNativeColumnarToRow : : : : : : +- CometBroadcastHashJoin : : : : : : :- CometFilter : : : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : : : : : +- SubqueryBroadcast - : : : : : : : +- BroadcastExchange - : : : : : : : +- CometNativeColumnarToRow - : : : : : : : +- CometProject - : : : : : : : +- CometFilter - : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : : +- CometSubqueryBroadcast + : : : : : : : +- CometBroadcastExchange + : : : : : : : +- CometProject + : : : : : : : +- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : : : +- CometBroadcastExchange : : : : : : +- CometBroadcastHashJoin : : : : : : :- CometFilter @@ -269,12 +257,11 @@ CometNativeColumnarToRow : : : : : : : +- CometBroadcastHashJoin : : : : : : : :- CometFilter : : : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : : : : : +- SubqueryBroadcast - : : : : : : : : +- BroadcastExchange - : : : : : : : : +- CometNativeColumnarToRow - : : : : : : : : +- CometProject - : : : : : : : : +- CometFilter - : : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : : : +- CometSubqueryBroadcast + : : : : : : : : +- CometBroadcastExchange + : : : : : : : : +- CometProject + : : : : : : : : +- CometFilter + : : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : : : : +- CometBroadcastExchange : : : : : : : +- CometFilter : : : : : : : +- CometNativeScan parquet spark_catalog.default.item @@ -293,12 +280,11 @@ CometNativeColumnarToRow : : : : : +- CometBroadcastHashJoin : : : : : :- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometNativeColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : +- CometSubqueryBroadcast + : : : : : : +- CometBroadcastExchange + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : : +- CometBroadcastExchange : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.item @@ -326,12 +312,11 @@ CometNativeColumnarToRow : : : : : +- CometBroadcastHashJoin : : : : : :- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometNativeColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : +- CometSubqueryBroadcast + : : : : : : +- CometBroadcastExchange + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : : +- CometBroadcastExchange : : : : : +- CometBroadcastHashJoin : : : : : :- CometFilter @@ -343,12 +328,11 @@ CometNativeColumnarToRow : : : : : : +- CometBroadcastHashJoin : : : : : : :- CometFilter : : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : : : : +- SubqueryBroadcast - : : : : : : : +- BroadcastExchange - : : : : : : : +- CometNativeColumnarToRow - : : : : : : : +- CometProject - : : : : : : : +- CometFilter - : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : : +- CometSubqueryBroadcast + : : : : : : : +- CometBroadcastExchange + : : : : : : : +- CometProject + : : : : : : : +- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : : : +- CometBroadcastExchange : : : : : : +- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.item @@ -367,12 +351,11 @@ CometNativeColumnarToRow : : : : +- CometBroadcastHashJoin : : : : :- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- CometSubqueryBroadcast + : : : : : +- CometBroadcastExchange + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : +- CometBroadcastExchange : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.item @@ -396,12 +379,11 @@ CometNativeColumnarToRow : : :- CometBroadcastHashJoin : : : :- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometSubqueryBroadcast + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometBroadcastHashJoin @@ -418,12 +400,11 @@ CometNativeColumnarToRow : : : : : +- CometBroadcastHashJoin : : : : : :- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometNativeColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : +- CometSubqueryBroadcast + : : : : : : +- CometBroadcastExchange + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : : +- CometBroadcastExchange : : : : : +- CometBroadcastHashJoin : : : : : :- CometFilter @@ -435,12 +416,11 @@ CometNativeColumnarToRow : : : : : : +- CometBroadcastHashJoin : : : : : : :- CometFilter : : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : : : : +- SubqueryBroadcast - : : : : : : : +- BroadcastExchange - : : : : : : : +- CometNativeColumnarToRow - : : : : : : : +- CometProject - : : : : : : : +- CometFilter - : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : : +- CometSubqueryBroadcast + : : : : : : : +- CometBroadcastExchange + : : : : : : : +- CometProject + : : : : : : : +- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : : : +- CometBroadcastExchange : : : : : : +- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.item @@ -459,12 +439,11 @@ CometNativeColumnarToRow : : : : +- CometBroadcastHashJoin : : : : :- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- CometSubqueryBroadcast + : : : : : +- CometBroadcastExchange + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : +- CometBroadcastExchange : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.item @@ -492,12 +471,11 @@ CometNativeColumnarToRow : : : : +- CometBroadcastHashJoin : : : : :- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- CometSubqueryBroadcast + : : : : : +- CometBroadcastExchange + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : +- CometBroadcastExchange : : : : +- CometBroadcastHashJoin : : : : :- CometFilter @@ -509,12 +487,11 @@ CometNativeColumnarToRow : : : : : +- CometBroadcastHashJoin : : : : : :- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometNativeColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : +- CometSubqueryBroadcast + : : : : : : +- CometBroadcastExchange + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : : +- CometBroadcastExchange : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.item @@ -533,12 +510,11 @@ CometNativeColumnarToRow : : : +- CometBroadcastHashJoin : : : :- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometSubqueryBroadcast + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- CometBroadcastExchange : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.item @@ -567,12 +543,11 @@ CometNativeColumnarToRow : : : :- CometProject : : : : +- CometBroadcastHashJoin : : : : :- CometNativeScan parquet spark_catalog.default.store_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- CometSubqueryBroadcast + : : : : : +- CometBroadcastExchange + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : +- CometBroadcastExchange : : : : +- CometProject : : : : +- CometFilter @@ -580,12 +555,11 @@ CometNativeColumnarToRow : : : :- CometProject : : : : +- CometBroadcastHashJoin : : : : :- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- CometSubqueryBroadcast + : : : : : +- CometBroadcastExchange + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : +- CometBroadcastExchange : : : : +- CometProject : : : : +- CometFilter @@ -593,12 +567,11 @@ CometNativeColumnarToRow : : : +- CometProject : : : +- CometBroadcastHashJoin : : : :- CometNativeScan parquet spark_catalog.default.web_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometSubqueryBroadcast + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter @@ -613,12 +586,11 @@ CometNativeColumnarToRow : : : :- CometBroadcastHashJoin : : : : :- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- CometSubqueryBroadcast + : : : : : +- CometBroadcastExchange + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : +- CometBroadcastExchange : : : : +- CometProject : : : : +- CometBroadcastHashJoin @@ -635,12 +607,11 @@ CometNativeColumnarToRow : : : : : : +- CometBroadcastHashJoin : : : : : : :- CometFilter : : : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : : : : : +- SubqueryBroadcast - : : : : : : : +- BroadcastExchange - : : : : : : : +- CometNativeColumnarToRow - : : : : : : : +- CometProject - : : : : : : : +- CometFilter - : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : : +- CometSubqueryBroadcast + : : : : : : : +- CometBroadcastExchange + : : : : : : : +- CometProject + : : : : : : : +- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : : : +- CometBroadcastExchange : : : : : : +- CometBroadcastHashJoin : : : : : : :- CometFilter @@ -652,12 +623,11 @@ CometNativeColumnarToRow : : : : : : : +- CometBroadcastHashJoin : : : : : : : :- CometFilter : : : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : : : : : +- SubqueryBroadcast - : : : : : : : : +- BroadcastExchange - : : : : : : : : +- CometNativeColumnarToRow - : : : : : : : : +- CometProject - : : : : : : : : +- CometFilter - : : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : : : +- CometSubqueryBroadcast + : : : : : : : : +- CometBroadcastExchange + : : : : : : : : +- CometProject + : : : : : : : : +- CometFilter + : : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : : : : +- CometBroadcastExchange : : : : : : : +- CometFilter : : : : : : : +- CometNativeScan parquet spark_catalog.default.item @@ -676,12 +646,11 @@ CometNativeColumnarToRow : : : : : +- CometBroadcastHashJoin : : : : : :- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometNativeColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : +- CometSubqueryBroadcast + : : : : : : +- CometBroadcastExchange + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : : +- CometBroadcastExchange : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.item @@ -709,12 +678,11 @@ CometNativeColumnarToRow : : : : : +- CometBroadcastHashJoin : : : : : :- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometNativeColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : +- CometSubqueryBroadcast + : : : : : : +- CometBroadcastExchange + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : : +- CometBroadcastExchange : : : : : +- CometBroadcastHashJoin : : : : : :- CometFilter @@ -726,12 +694,11 @@ CometNativeColumnarToRow : : : : : : +- CometBroadcastHashJoin : : : : : : :- CometFilter : : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : : : : +- SubqueryBroadcast - : : : : : : : +- BroadcastExchange - : : : : : : : +- CometNativeColumnarToRow - : : : : : : : +- CometProject - : : : : : : : +- CometFilter - : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : : +- CometSubqueryBroadcast + : : : : : : : +- CometBroadcastExchange + : : : : : : : +- CometProject + : : : : : : : +- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : : : +- CometBroadcastExchange : : : : : : +- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.item @@ -750,12 +717,11 @@ CometNativeColumnarToRow : : : : +- CometBroadcastHashJoin : : : : :- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- CometSubqueryBroadcast + : : : : : +- CometBroadcastExchange + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : +- CometBroadcastExchange : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.item @@ -779,12 +745,11 @@ CometNativeColumnarToRow : : : :- CometBroadcastHashJoin : : : : :- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- CometSubqueryBroadcast + : : : : : +- CometBroadcastExchange + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : +- CometBroadcastExchange : : : : +- CometProject : : : : +- CometBroadcastHashJoin @@ -801,12 +766,11 @@ CometNativeColumnarToRow : : : : : : +- CometBroadcastHashJoin : : : : : : :- CometFilter : : : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : : : : : +- SubqueryBroadcast - : : : : : : : +- BroadcastExchange - : : : : : : : +- CometNativeColumnarToRow - : : : : : : : +- CometProject - : : : : : : : +- CometFilter - : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : : +- CometSubqueryBroadcast + : : : : : : : +- CometBroadcastExchange + : : : : : : : +- CometProject + : : : : : : : +- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : : : +- CometBroadcastExchange : : : : : : +- CometBroadcastHashJoin : : : : : : :- CometFilter @@ -818,12 +782,11 @@ CometNativeColumnarToRow : : : : : : : +- CometBroadcastHashJoin : : : : : : : :- CometFilter : : : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : : : : : +- SubqueryBroadcast - : : : : : : : : +- BroadcastExchange - : : : : : : : : +- CometNativeColumnarToRow - : : : : : : : : +- CometProject - : : : : : : : : +- CometFilter - : : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : : : +- CometSubqueryBroadcast + : : : : : : : : +- CometBroadcastExchange + : : : : : : : : +- CometProject + : : : : : : : : +- CometFilter + : : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : : : : +- CometBroadcastExchange : : : : : : : +- CometFilter : : : : : : : +- CometNativeScan parquet spark_catalog.default.item @@ -842,12 +805,11 @@ CometNativeColumnarToRow : : : : : +- CometBroadcastHashJoin : : : : : :- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometNativeColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : +- CometSubqueryBroadcast + : : : : : : +- CometBroadcastExchange + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : : +- CometBroadcastExchange : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.item @@ -875,12 +837,11 @@ CometNativeColumnarToRow : : : : : +- CometBroadcastHashJoin : : : : : :- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometNativeColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : +- CometSubqueryBroadcast + : : : : : : +- CometBroadcastExchange + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : : +- CometBroadcastExchange : : : : : +- CometBroadcastHashJoin : : : : : :- CometFilter @@ -892,12 +853,11 @@ CometNativeColumnarToRow : : : : : : +- CometBroadcastHashJoin : : : : : : :- CometFilter : : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : : : : +- SubqueryBroadcast - : : : : : : : +- BroadcastExchange - : : : : : : : +- CometNativeColumnarToRow - : : : : : : : +- CometProject - : : : : : : : +- CometFilter - : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : : +- CometSubqueryBroadcast + : : : : : : : +- CometBroadcastExchange + : : : : : : : +- CometProject + : : : : : : : +- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : : : +- CometBroadcastExchange : : : : : : +- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.item @@ -916,12 +876,11 @@ CometNativeColumnarToRow : : : : +- CometBroadcastHashJoin : : : : :- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- CometSubqueryBroadcast + : : : : : +- CometBroadcastExchange + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : +- CometBroadcastExchange : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.item @@ -945,12 +904,11 @@ CometNativeColumnarToRow : : :- CometBroadcastHashJoin : : : :- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometSubqueryBroadcast + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometBroadcastHashJoin @@ -967,12 +925,11 @@ CometNativeColumnarToRow : : : : : +- CometBroadcastHashJoin : : : : : :- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometNativeColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : +- CometSubqueryBroadcast + : : : : : : +- CometBroadcastExchange + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : : +- CometBroadcastExchange : : : : : +- CometBroadcastHashJoin : : : : : :- CometFilter @@ -984,12 +941,11 @@ CometNativeColumnarToRow : : : : : : +- CometBroadcastHashJoin : : : : : : :- CometFilter : : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : : : : +- SubqueryBroadcast - : : : : : : : +- BroadcastExchange - : : : : : : : +- CometNativeColumnarToRow - : : : : : : : +- CometProject - : : : : : : : +- CometFilter - : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : : +- CometSubqueryBroadcast + : : : : : : : +- CometBroadcastExchange + : : : : : : : +- CometProject + : : : : : : : +- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : : : +- CometBroadcastExchange : : : : : : +- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.item @@ -1008,12 +964,11 @@ CometNativeColumnarToRow : : : : +- CometBroadcastHashJoin : : : : :- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- CometSubqueryBroadcast + : : : : : +- CometBroadcastExchange + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : +- CometBroadcastExchange : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.item @@ -1041,12 +996,11 @@ CometNativeColumnarToRow : : : : +- CometBroadcastHashJoin : : : : :- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- CometSubqueryBroadcast + : : : : : +- CometBroadcastExchange + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : +- CometBroadcastExchange : : : : +- CometBroadcastHashJoin : : : : :- CometFilter @@ -1058,12 +1012,11 @@ CometNativeColumnarToRow : : : : : +- CometBroadcastHashJoin : : : : : :- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometNativeColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : +- CometSubqueryBroadcast + : : : : : : +- CometBroadcastExchange + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : : +- CometBroadcastExchange : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.item @@ -1082,12 +1035,11 @@ CometNativeColumnarToRow : : : +- CometBroadcastHashJoin : : : :- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometSubqueryBroadcast + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- CometBroadcastExchange : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.item @@ -1116,12 +1068,11 @@ CometNativeColumnarToRow : : : :- CometProject : : : : +- CometBroadcastHashJoin : : : : :- CometNativeScan parquet spark_catalog.default.store_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- CometSubqueryBroadcast + : : : : : +- CometBroadcastExchange + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : +- CometBroadcastExchange : : : : +- CometProject : : : : +- CometFilter @@ -1129,12 +1080,11 @@ CometNativeColumnarToRow : : : :- CometProject : : : : +- CometBroadcastHashJoin : : : : :- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- CometSubqueryBroadcast + : : : : : +- CometBroadcastExchange + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : +- CometBroadcastExchange : : : : +- CometProject : : : : +- CometFilter @@ -1142,12 +1092,11 @@ CometNativeColumnarToRow : : : +- CometProject : : : +- CometBroadcastHashJoin : : : :- CometNativeScan parquet spark_catalog.default.web_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometSubqueryBroadcast + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter @@ -1162,12 +1111,11 @@ CometNativeColumnarToRow : : : :- CometBroadcastHashJoin : : : : :- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- CometSubqueryBroadcast + : : : : : +- CometBroadcastExchange + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : +- CometBroadcastExchange : : : : +- CometProject : : : : +- CometBroadcastHashJoin @@ -1184,12 +1132,11 @@ CometNativeColumnarToRow : : : : : : +- CometBroadcastHashJoin : : : : : : :- CometFilter : : : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : : : : : +- SubqueryBroadcast - : : : : : : : +- BroadcastExchange - : : : : : : : +- CometNativeColumnarToRow - : : : : : : : +- CometProject - : : : : : : : +- CometFilter - : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : : +- CometSubqueryBroadcast + : : : : : : : +- CometBroadcastExchange + : : : : : : : +- CometProject + : : : : : : : +- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : : : +- CometBroadcastExchange : : : : : : +- CometBroadcastHashJoin : : : : : : :- CometFilter @@ -1201,12 +1148,11 @@ CometNativeColumnarToRow : : : : : : : +- CometBroadcastHashJoin : : : : : : : :- CometFilter : : : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : : : : : +- SubqueryBroadcast - : : : : : : : : +- BroadcastExchange - : : : : : : : : +- CometNativeColumnarToRow - : : : : : : : : +- CometProject - : : : : : : : : +- CometFilter - : : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : : : +- CometSubqueryBroadcast + : : : : : : : : +- CometBroadcastExchange + : : : : : : : : +- CometProject + : : : : : : : : +- CometFilter + : : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : : : : +- CometBroadcastExchange : : : : : : : +- CometFilter : : : : : : : +- CometNativeScan parquet spark_catalog.default.item @@ -1225,12 +1171,11 @@ CometNativeColumnarToRow : : : : : +- CometBroadcastHashJoin : : : : : :- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometNativeColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : +- CometSubqueryBroadcast + : : : : : : +- CometBroadcastExchange + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : : +- CometBroadcastExchange : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.item @@ -1258,12 +1203,11 @@ CometNativeColumnarToRow : : : : : +- CometBroadcastHashJoin : : : : : :- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometNativeColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : +- CometSubqueryBroadcast + : : : : : : +- CometBroadcastExchange + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : : +- CometBroadcastExchange : : : : : +- CometBroadcastHashJoin : : : : : :- CometFilter @@ -1275,12 +1219,11 @@ CometNativeColumnarToRow : : : : : : +- CometBroadcastHashJoin : : : : : : :- CometFilter : : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : : : : +- SubqueryBroadcast - : : : : : : : +- BroadcastExchange - : : : : : : : +- CometNativeColumnarToRow - : : : : : : : +- CometProject - : : : : : : : +- CometFilter - : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : : +- CometSubqueryBroadcast + : : : : : : : +- CometBroadcastExchange + : : : : : : : +- CometProject + : : : : : : : +- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : : : +- CometBroadcastExchange : : : : : : +- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.item @@ -1299,12 +1242,11 @@ CometNativeColumnarToRow : : : : +- CometBroadcastHashJoin : : : : :- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- CometSubqueryBroadcast + : : : : : +- CometBroadcastExchange + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : +- CometBroadcastExchange : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.item @@ -1328,12 +1270,11 @@ CometNativeColumnarToRow : : : :- CometBroadcastHashJoin : : : : :- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- CometSubqueryBroadcast + : : : : : +- CometBroadcastExchange + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : +- CometBroadcastExchange : : : : +- CometProject : : : : +- CometBroadcastHashJoin @@ -1350,12 +1291,11 @@ CometNativeColumnarToRow : : : : : : +- CometBroadcastHashJoin : : : : : : :- CometFilter : : : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : : : : : +- SubqueryBroadcast - : : : : : : : +- BroadcastExchange - : : : : : : : +- CometNativeColumnarToRow - : : : : : : : +- CometProject - : : : : : : : +- CometFilter - : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : : +- CometSubqueryBroadcast + : : : : : : : +- CometBroadcastExchange + : : : : : : : +- CometProject + : : : : : : : +- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : : : +- CometBroadcastExchange : : : : : : +- CometBroadcastHashJoin : : : : : : :- CometFilter @@ -1367,12 +1307,11 @@ CometNativeColumnarToRow : : : : : : : +- CometBroadcastHashJoin : : : : : : : :- CometFilter : : : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : : : : : +- SubqueryBroadcast - : : : : : : : : +- BroadcastExchange - : : : : : : : : +- CometNativeColumnarToRow - : : : : : : : : +- CometProject - : : : : : : : : +- CometFilter - : : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : : : +- CometSubqueryBroadcast + : : : : : : : : +- CometBroadcastExchange + : : : : : : : : +- CometProject + : : : : : : : : +- CometFilter + : : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : : : : +- CometBroadcastExchange : : : : : : : +- CometFilter : : : : : : : +- CometNativeScan parquet spark_catalog.default.item @@ -1391,12 +1330,11 @@ CometNativeColumnarToRow : : : : : +- CometBroadcastHashJoin : : : : : :- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometNativeColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : +- CometSubqueryBroadcast + : : : : : : +- CometBroadcastExchange + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : : +- CometBroadcastExchange : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.item @@ -1424,12 +1362,11 @@ CometNativeColumnarToRow : : : : : +- CometBroadcastHashJoin : : : : : :- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometNativeColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : +- CometSubqueryBroadcast + : : : : : : +- CometBroadcastExchange + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : : +- CometBroadcastExchange : : : : : +- CometBroadcastHashJoin : : : : : :- CometFilter @@ -1441,12 +1378,11 @@ CometNativeColumnarToRow : : : : : : +- CometBroadcastHashJoin : : : : : : :- CometFilter : : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : : : : +- SubqueryBroadcast - : : : : : : : +- BroadcastExchange - : : : : : : : +- CometNativeColumnarToRow - : : : : : : : +- CometProject - : : : : : : : +- CometFilter - : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : : +- CometSubqueryBroadcast + : : : : : : : +- CometBroadcastExchange + : : : : : : : +- CometProject + : : : : : : : +- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : : : +- CometBroadcastExchange : : : : : : +- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.item @@ -1465,12 +1401,11 @@ CometNativeColumnarToRow : : : : +- CometBroadcastHashJoin : : : : :- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- CometSubqueryBroadcast + : : : : : +- CometBroadcastExchange + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : +- CometBroadcastExchange : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.item @@ -1494,12 +1429,11 @@ CometNativeColumnarToRow : : :- CometBroadcastHashJoin : : : :- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometSubqueryBroadcast + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometBroadcastHashJoin @@ -1516,12 +1450,11 @@ CometNativeColumnarToRow : : : : : +- CometBroadcastHashJoin : : : : : :- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometNativeColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : +- CometSubqueryBroadcast + : : : : : : +- CometBroadcastExchange + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : : +- CometBroadcastExchange : : : : : +- CometBroadcastHashJoin : : : : : :- CometFilter @@ -1533,12 +1466,11 @@ CometNativeColumnarToRow : : : : : : +- CometBroadcastHashJoin : : : : : : :- CometFilter : : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : : : : +- SubqueryBroadcast - : : : : : : : +- BroadcastExchange - : : : : : : : +- CometNativeColumnarToRow - : : : : : : : +- CometProject - : : : : : : : +- CometFilter - : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : : +- CometSubqueryBroadcast + : : : : : : : +- CometBroadcastExchange + : : : : : : : +- CometProject + : : : : : : : +- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : : : +- CometBroadcastExchange : : : : : : +- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.item @@ -1557,12 +1489,11 @@ CometNativeColumnarToRow : : : : +- CometBroadcastHashJoin : : : : :- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- CometSubqueryBroadcast + : : : : : +- CometBroadcastExchange + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : +- CometBroadcastExchange : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.item @@ -1590,12 +1521,11 @@ CometNativeColumnarToRow : : : : +- CometBroadcastHashJoin : : : : :- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- CometSubqueryBroadcast + : : : : : +- CometBroadcastExchange + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : +- CometBroadcastExchange : : : : +- CometBroadcastHashJoin : : : : :- CometFilter @@ -1607,12 +1537,11 @@ CometNativeColumnarToRow : : : : : +- CometBroadcastHashJoin : : : : : :- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometNativeColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : +- CometSubqueryBroadcast + : : : : : : +- CometBroadcastExchange + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : : +- CometBroadcastExchange : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.item @@ -1631,12 +1560,11 @@ CometNativeColumnarToRow : : : +- CometBroadcastHashJoin : : : :- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometSubqueryBroadcast + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- CometBroadcastExchange : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.item @@ -1665,12 +1593,11 @@ CometNativeColumnarToRow : : : :- CometProject : : : : +- CometBroadcastHashJoin : : : : :- CometNativeScan parquet spark_catalog.default.store_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- CometSubqueryBroadcast + : : : : : +- CometBroadcastExchange + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : +- CometBroadcastExchange : : : : +- CometProject : : : : +- CometFilter @@ -1678,12 +1605,11 @@ CometNativeColumnarToRow : : : :- CometProject : : : : +- CometBroadcastHashJoin : : : : :- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- CometSubqueryBroadcast + : : : : : +- CometBroadcastExchange + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : +- CometBroadcastExchange : : : : +- CometProject : : : : +- CometFilter @@ -1691,12 +1617,11 @@ CometNativeColumnarToRow : : : +- CometProject : : : +- CometBroadcastHashJoin : : : :- CometNativeScan parquet spark_catalog.default.web_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometSubqueryBroadcast + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter @@ -1711,12 +1636,11 @@ CometNativeColumnarToRow : : : :- CometBroadcastHashJoin : : : : :- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- CometSubqueryBroadcast + : : : : : +- CometBroadcastExchange + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : +- CometBroadcastExchange : : : : +- CometProject : : : : +- CometBroadcastHashJoin @@ -1733,12 +1657,11 @@ CometNativeColumnarToRow : : : : : : +- CometBroadcastHashJoin : : : : : : :- CometFilter : : : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : : : : : +- SubqueryBroadcast - : : : : : : : +- BroadcastExchange - : : : : : : : +- CometNativeColumnarToRow - : : : : : : : +- CometProject - : : : : : : : +- CometFilter - : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : : +- CometSubqueryBroadcast + : : : : : : : +- CometBroadcastExchange + : : : : : : : +- CometProject + : : : : : : : +- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : : : +- CometBroadcastExchange : : : : : : +- CometBroadcastHashJoin : : : : : : :- CometFilter @@ -1750,12 +1673,11 @@ CometNativeColumnarToRow : : : : : : : +- CometBroadcastHashJoin : : : : : : : :- CometFilter : : : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : : : : : +- SubqueryBroadcast - : : : : : : : : +- BroadcastExchange - : : : : : : : : +- CometNativeColumnarToRow - : : : : : : : : +- CometProject - : : : : : : : : +- CometFilter - : : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : : : +- CometSubqueryBroadcast + : : : : : : : : +- CometBroadcastExchange + : : : : : : : : +- CometProject + : : : : : : : : +- CometFilter + : : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : : : : +- CometBroadcastExchange : : : : : : : +- CometFilter : : : : : : : +- CometNativeScan parquet spark_catalog.default.item @@ -1774,12 +1696,11 @@ CometNativeColumnarToRow : : : : : +- CometBroadcastHashJoin : : : : : :- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometNativeColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : +- CometSubqueryBroadcast + : : : : : : +- CometBroadcastExchange + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : : +- CometBroadcastExchange : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.item @@ -1807,12 +1728,11 @@ CometNativeColumnarToRow : : : : : +- CometBroadcastHashJoin : : : : : :- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometNativeColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : +- CometSubqueryBroadcast + : : : : : : +- CometBroadcastExchange + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : : +- CometBroadcastExchange : : : : : +- CometBroadcastHashJoin : : : : : :- CometFilter @@ -1824,12 +1744,11 @@ CometNativeColumnarToRow : : : : : : +- CometBroadcastHashJoin : : : : : : :- CometFilter : : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : : : : +- SubqueryBroadcast - : : : : : : : +- BroadcastExchange - : : : : : : : +- CometNativeColumnarToRow - : : : : : : : +- CometProject - : : : : : : : +- CometFilter - : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : : +- CometSubqueryBroadcast + : : : : : : : +- CometBroadcastExchange + : : : : : : : +- CometProject + : : : : : : : +- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : : : +- CometBroadcastExchange : : : : : : +- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.item @@ -1848,12 +1767,11 @@ CometNativeColumnarToRow : : : : +- CometBroadcastHashJoin : : : : :- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- CometSubqueryBroadcast + : : : : : +- CometBroadcastExchange + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : +- CometBroadcastExchange : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.item @@ -1877,12 +1795,11 @@ CometNativeColumnarToRow : : : :- CometBroadcastHashJoin : : : : :- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- CometSubqueryBroadcast + : : : : : +- CometBroadcastExchange + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : +- CometBroadcastExchange : : : : +- CometProject : : : : +- CometBroadcastHashJoin @@ -1899,12 +1816,11 @@ CometNativeColumnarToRow : : : : : : +- CometBroadcastHashJoin : : : : : : :- CometFilter : : : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : : : : : +- SubqueryBroadcast - : : : : : : : +- BroadcastExchange - : : : : : : : +- CometNativeColumnarToRow - : : : : : : : +- CometProject - : : : : : : : +- CometFilter - : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : : +- CometSubqueryBroadcast + : : : : : : : +- CometBroadcastExchange + : : : : : : : +- CometProject + : : : : : : : +- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : : : +- CometBroadcastExchange : : : : : : +- CometBroadcastHashJoin : : : : : : :- CometFilter @@ -1916,12 +1832,11 @@ CometNativeColumnarToRow : : : : : : : +- CometBroadcastHashJoin : : : : : : : :- CometFilter : : : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : : : : : +- SubqueryBroadcast - : : : : : : : : +- BroadcastExchange - : : : : : : : : +- CometNativeColumnarToRow - : : : : : : : : +- CometProject - : : : : : : : : +- CometFilter - : : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : : : +- CometSubqueryBroadcast + : : : : : : : : +- CometBroadcastExchange + : : : : : : : : +- CometProject + : : : : : : : : +- CometFilter + : : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : : : : +- CometBroadcastExchange : : : : : : : +- CometFilter : : : : : : : +- CometNativeScan parquet spark_catalog.default.item @@ -1940,12 +1855,11 @@ CometNativeColumnarToRow : : : : : +- CometBroadcastHashJoin : : : : : :- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometNativeColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : +- CometSubqueryBroadcast + : : : : : : +- CometBroadcastExchange + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : : +- CometBroadcastExchange : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.item @@ -1973,12 +1887,11 @@ CometNativeColumnarToRow : : : : : +- CometBroadcastHashJoin : : : : : :- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometNativeColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : +- CometSubqueryBroadcast + : : : : : : +- CometBroadcastExchange + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : : +- CometBroadcastExchange : : : : : +- CometBroadcastHashJoin : : : : : :- CometFilter @@ -1990,12 +1903,11 @@ CometNativeColumnarToRow : : : : : : +- CometBroadcastHashJoin : : : : : : :- CometFilter : : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : : : : +- SubqueryBroadcast - : : : : : : : +- BroadcastExchange - : : : : : : : +- CometNativeColumnarToRow - : : : : : : : +- CometProject - : : : : : : : +- CometFilter - : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : : +- CometSubqueryBroadcast + : : : : : : : +- CometBroadcastExchange + : : : : : : : +- CometProject + : : : : : : : +- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : : : +- CometBroadcastExchange : : : : : : +- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.item @@ -2014,12 +1926,11 @@ CometNativeColumnarToRow : : : : +- CometBroadcastHashJoin : : : : :- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- CometSubqueryBroadcast + : : : : : +- CometBroadcastExchange + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : +- CometBroadcastExchange : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.item @@ -2043,12 +1954,11 @@ CometNativeColumnarToRow : : :- CometBroadcastHashJoin : : : :- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometSubqueryBroadcast + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometBroadcastHashJoin @@ -2065,12 +1975,11 @@ CometNativeColumnarToRow : : : : : +- CometBroadcastHashJoin : : : : : :- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometNativeColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : +- CometSubqueryBroadcast + : : : : : : +- CometBroadcastExchange + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : : +- CometBroadcastExchange : : : : : +- CometBroadcastHashJoin : : : : : :- CometFilter @@ -2082,12 +1991,11 @@ CometNativeColumnarToRow : : : : : : +- CometBroadcastHashJoin : : : : : : :- CometFilter : : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : : : : +- SubqueryBroadcast - : : : : : : : +- BroadcastExchange - : : : : : : : +- CometNativeColumnarToRow - : : : : : : : +- CometProject - : : : : : : : +- CometFilter - : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : : +- CometSubqueryBroadcast + : : : : : : : +- CometBroadcastExchange + : : : : : : : +- CometProject + : : : : : : : +- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : : : +- CometBroadcastExchange : : : : : : +- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.item @@ -2106,12 +2014,11 @@ CometNativeColumnarToRow : : : : +- CometBroadcastHashJoin : : : : :- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- CometSubqueryBroadcast + : : : : : +- CometBroadcastExchange + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : +- CometBroadcastExchange : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.item @@ -2139,12 +2046,11 @@ CometNativeColumnarToRow : : : : +- CometBroadcastHashJoin : : : : :- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- CometSubqueryBroadcast + : : : : : +- CometBroadcastExchange + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : +- CometBroadcastExchange : : : : +- CometBroadcastHashJoin : : : : :- CometFilter @@ -2156,12 +2062,11 @@ CometNativeColumnarToRow : : : : : +- CometBroadcastHashJoin : : : : : :- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometNativeColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : +- CometSubqueryBroadcast + : : : : : : +- CometBroadcastExchange + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : : +- CometBroadcastExchange : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.item @@ -2180,12 +2085,11 @@ CometNativeColumnarToRow : : : +- CometBroadcastHashJoin : : : :- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometSubqueryBroadcast + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- CometBroadcastExchange : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.item @@ -2214,12 +2118,11 @@ CometNativeColumnarToRow : : :- CometProject : : : +- CometBroadcastHashJoin : : : :- CometNativeScan parquet spark_catalog.default.store_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometSubqueryBroadcast + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter @@ -2227,12 +2130,11 @@ CometNativeColumnarToRow : : :- CometProject : : : +- CometBroadcastHashJoin : : : :- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometSubqueryBroadcast + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter @@ -2240,12 +2142,11 @@ CometNativeColumnarToRow : : +- CometProject : : +- CometBroadcastHashJoin : : :- CometNativeScan parquet spark_catalog.default.web_sales - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometSubqueryBroadcast + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- CometBroadcastExchange : : +- CometProject : : +- CometFilter @@ -2260,12 +2161,11 @@ CometNativeColumnarToRow : : :- CometBroadcastHashJoin : : : :- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometSubqueryBroadcast + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometBroadcastHashJoin @@ -2282,12 +2182,11 @@ CometNativeColumnarToRow : : : : : +- CometBroadcastHashJoin : : : : : :- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometNativeColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : +- CometSubqueryBroadcast + : : : : : : +- CometBroadcastExchange + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : : +- CometBroadcastExchange : : : : : +- CometBroadcastHashJoin : : : : : :- CometFilter @@ -2299,12 +2198,11 @@ CometNativeColumnarToRow : : : : : : +- CometBroadcastHashJoin : : : : : : :- CometFilter : : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : : : : +- SubqueryBroadcast - : : : : : : : +- BroadcastExchange - : : : : : : : +- CometNativeColumnarToRow - : : : : : : : +- CometProject - : : : : : : : +- CometFilter - : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : : +- CometSubqueryBroadcast + : : : : : : : +- CometBroadcastExchange + : : : : : : : +- CometProject + : : : : : : : +- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : : : +- CometBroadcastExchange : : : : : : +- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.item @@ -2323,12 +2221,11 @@ CometNativeColumnarToRow : : : : +- CometBroadcastHashJoin : : : : :- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- CometSubqueryBroadcast + : : : : : +- CometBroadcastExchange + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : +- CometBroadcastExchange : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.item @@ -2356,12 +2253,11 @@ CometNativeColumnarToRow : : : : +- CometBroadcastHashJoin : : : : :- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- CometSubqueryBroadcast + : : : : : +- CometBroadcastExchange + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : +- CometBroadcastExchange : : : : +- CometBroadcastHashJoin : : : : :- CometFilter @@ -2373,12 +2269,11 @@ CometNativeColumnarToRow : : : : : +- CometBroadcastHashJoin : : : : : :- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometNativeColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : +- CometSubqueryBroadcast + : : : : : : +- CometBroadcastExchange + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : : +- CometBroadcastExchange : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.item @@ -2397,12 +2292,11 @@ CometNativeColumnarToRow : : : +- CometBroadcastHashJoin : : : :- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometSubqueryBroadcast + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- CometBroadcastExchange : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.item @@ -2426,12 +2320,11 @@ CometNativeColumnarToRow : : :- CometBroadcastHashJoin : : : :- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometSubqueryBroadcast + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometBroadcastHashJoin @@ -2448,12 +2341,11 @@ CometNativeColumnarToRow : : : : : +- CometBroadcastHashJoin : : : : : :- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometNativeColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : +- CometSubqueryBroadcast + : : : : : : +- CometBroadcastExchange + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : : +- CometBroadcastExchange : : : : : +- CometBroadcastHashJoin : : : : : :- CometFilter @@ -2465,12 +2357,11 @@ CometNativeColumnarToRow : : : : : : +- CometBroadcastHashJoin : : : : : : :- CometFilter : : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : : : : +- SubqueryBroadcast - : : : : : : : +- BroadcastExchange - : : : : : : : +- CometNativeColumnarToRow - : : : : : : : +- CometProject - : : : : : : : +- CometFilter - : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : : +- CometSubqueryBroadcast + : : : : : : : +- CometBroadcastExchange + : : : : : : : +- CometProject + : : : : : : : +- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : : : +- CometBroadcastExchange : : : : : : +- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.item @@ -2489,12 +2380,11 @@ CometNativeColumnarToRow : : : : +- CometBroadcastHashJoin : : : : :- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- CometSubqueryBroadcast + : : : : : +- CometBroadcastExchange + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : +- CometBroadcastExchange : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.item @@ -2522,12 +2412,11 @@ CometNativeColumnarToRow : : : : +- CometBroadcastHashJoin : : : : :- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- CometSubqueryBroadcast + : : : : : +- CometBroadcastExchange + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : +- CometBroadcastExchange : : : : +- CometBroadcastHashJoin : : : : :- CometFilter @@ -2539,12 +2428,11 @@ CometNativeColumnarToRow : : : : : +- CometBroadcastHashJoin : : : : : :- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometNativeColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : +- CometSubqueryBroadcast + : : : : : : +- CometBroadcastExchange + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : : +- CometBroadcastExchange : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.item @@ -2563,12 +2451,11 @@ CometNativeColumnarToRow : : : +- CometBroadcastHashJoin : : : :- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometSubqueryBroadcast + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- CometBroadcastExchange : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.item @@ -2592,12 +2479,11 @@ CometNativeColumnarToRow : :- CometBroadcastHashJoin : : :- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometSubqueryBroadcast + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- CometBroadcastExchange : : +- CometProject : : +- CometBroadcastHashJoin @@ -2614,12 +2500,11 @@ CometNativeColumnarToRow : : : : +- CometBroadcastHashJoin : : : : :- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- CometSubqueryBroadcast + : : : : : +- CometBroadcastExchange + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : +- CometBroadcastExchange : : : : +- CometBroadcastHashJoin : : : : :- CometFilter @@ -2631,12 +2516,11 @@ CometNativeColumnarToRow : : : : : +- CometBroadcastHashJoin : : : : : :- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometNativeColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : +- CometSubqueryBroadcast + : : : : : : +- CometBroadcastExchange + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : : +- CometBroadcastExchange : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.item @@ -2655,12 +2539,11 @@ CometNativeColumnarToRow : : : +- CometBroadcastHashJoin : : : :- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometSubqueryBroadcast + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- CometBroadcastExchange : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.item @@ -2688,12 +2571,11 @@ CometNativeColumnarToRow : : : +- CometBroadcastHashJoin : : : :- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometSubqueryBroadcast + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- CometBroadcastExchange : : : +- CometBroadcastHashJoin : : : :- CometFilter @@ -2705,12 +2587,11 @@ CometNativeColumnarToRow : : : : +- CometBroadcastHashJoin : : : : :- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- CometSubqueryBroadcast + : : : : : +- CometBroadcastExchange + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : +- CometBroadcastExchange : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.item @@ -2729,12 +2610,11 @@ CometNativeColumnarToRow : : +- CometBroadcastHashJoin : : :- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometSubqueryBroadcast + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- CometBroadcastExchange : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.item @@ -2747,4 +2627,4 @@ CometNativeColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.date_dim -Comet accelerated 2367 out of 2622 eligible operators (90%). Final plan contains 126 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 2487 out of 2622 eligible operators (94%). Final plan contains 6 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q18a.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q18a.native_datafusion/extended.txt index 82bacea2fb..b34ebfdf84 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q18a.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q18a.native_datafusion/extended.txt @@ -18,12 +18,11 @@ CometNativeColumnarToRow : : : : : : +- CometBroadcastHashJoin : : : : : : :- CometFilter : : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : : : : +- SubqueryBroadcast - : : : : : : : +- BroadcastExchange - : : : : : : : +- CometNativeColumnarToRow - : : : : : : : +- CometProject - : : : : : : : +- CometFilter - : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : : +- CometSubqueryBroadcast + : : : : : : : +- CometBroadcastExchange + : : : : : : : +- CometProject + : : : : : : : +- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : : : +- CometBroadcastExchange : : : : : : +- CometProject : : : : : : +- CometFilter @@ -64,12 +63,11 @@ CometNativeColumnarToRow : : : : : : +- CometBroadcastHashJoin : : : : : : :- CometFilter : : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : : : : +- SubqueryBroadcast - : : : : : : : +- BroadcastExchange - : : : : : : : +- CometNativeColumnarToRow - : : : : : : : +- CometProject - : : : : : : : +- CometFilter - : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : : +- CometSubqueryBroadcast + : : : : : : : +- CometBroadcastExchange + : : : : : : : +- CometProject + : : : : : : : +- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : : : +- CometBroadcastExchange : : : : : : +- CometProject : : : : : : +- CometFilter @@ -110,12 +108,11 @@ CometNativeColumnarToRow : : : : : : +- CometBroadcastHashJoin : : : : : : :- CometFilter : : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : : : : +- SubqueryBroadcast - : : : : : : : +- BroadcastExchange - : : : : : : : +- CometNativeColumnarToRow - : : : : : : : +- CometProject - : : : : : : : +- CometFilter - : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : : +- CometSubqueryBroadcast + : : : : : : : +- CometBroadcastExchange + : : : : : : : +- CometProject + : : : : : : : +- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : : : +- CometBroadcastExchange : : : : : : +- CometProject : : : : : : +- CometFilter @@ -156,12 +153,11 @@ CometNativeColumnarToRow : : : : : : +- CometBroadcastHashJoin : : : : : : :- CometFilter : : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : : : : +- SubqueryBroadcast - : : : : : : : +- BroadcastExchange - : : : : : : : +- CometNativeColumnarToRow - : : : : : : : +- CometProject - : : : : : : : +- CometFilter - : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : : +- CometSubqueryBroadcast + : : : : : : : +- CometBroadcastExchange + : : : : : : : +- CometProject + : : : : : : : +- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : : : +- CometBroadcastExchange : : : : : : +- CometProject : : : : : : +- CometFilter @@ -202,12 +198,11 @@ CometNativeColumnarToRow : : : : : +- CometBroadcastHashJoin : : : : : :- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometNativeColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : +- CometSubqueryBroadcast + : : : : : : +- CometBroadcastExchange + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : : +- CometBroadcastExchange : : : : : +- CometProject : : : : : +- CometFilter @@ -231,4 +226,4 @@ CometNativeColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.item -Comet accelerated 216 out of 226 eligible operators (95%). Final plan contains 6 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 221 out of 226 eligible operators (97%). Final plan contains 1 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q20.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q20.native_datafusion/extended.txt index c4f54f4289..24498162ca 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q20.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q20.native_datafusion/extended.txt @@ -13,12 +13,11 @@ TakeOrderedAndProject : +- CometBroadcastHashJoin : :- CometFilter : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometSubqueryBroadcast + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim : +- CometBroadcastExchange : +- CometProject : +- CometFilter @@ -28,4 +27,4 @@ TakeOrderedAndProject +- CometFilter +- CometNativeScan parquet spark_catalog.default.date_dim -Comet accelerated 22 out of 27 eligible operators (81%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 23 out of 27 eligible operators (85%). Final plan contains 1 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q22.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q22.native_datafusion/extended.txt index a8588fd031..5d9c8333d0 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q22.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q22.native_datafusion/extended.txt @@ -13,12 +13,11 @@ TakeOrderedAndProject : : +- CometBroadcastHashJoin : : :- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.inventory - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometSubqueryBroadcast + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- CometBroadcastExchange : : +- CometProject : : +- CometFilter @@ -31,4 +30,4 @@ TakeOrderedAndProject +- CometNativeColumnarToRow +- CometNativeScan parquet spark_catalog.default.warehouse -Comet accelerated 19 out of 28 eligible operators (67%). Final plan contains 4 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 20 out of 28 eligible operators (71%). Final plan contains 3 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q22a.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q22a.native_datafusion/extended.txt index 5bd5daea84..0d8c15e00c 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q22a.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q22a.native_datafusion/extended.txt @@ -14,12 +14,11 @@ CometNativeColumnarToRow : : : +- CometBroadcastHashJoin : : : :- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.inventory - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometSubqueryBroadcast + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter @@ -45,12 +44,11 @@ CometNativeColumnarToRow : : : +- CometBroadcastHashJoin : : : :- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.inventory - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometSubqueryBroadcast + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter @@ -76,12 +74,11 @@ CometNativeColumnarToRow : : : +- CometBroadcastHashJoin : : : :- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.inventory - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometSubqueryBroadcast + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter @@ -107,12 +104,11 @@ CometNativeColumnarToRow : : : +- CometBroadcastHashJoin : : : :- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.inventory - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometSubqueryBroadcast + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter @@ -138,12 +134,11 @@ CometNativeColumnarToRow : : +- CometBroadcastHashJoin : : :- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.inventory - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometSubqueryBroadcast + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- CometBroadcastExchange : : +- CometProject : : +- CometFilter @@ -156,4 +151,4 @@ CometNativeColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.warehouse -Comet accelerated 141 out of 151 eligible operators (93%). Final plan contains 6 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 146 out of 151 eligible operators (96%). Final plan contains 1 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q27a.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q27a.native_datafusion/extended.txt index 5dc06ab86f..9c4497cbd3 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q27a.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q27a.native_datafusion/extended.txt @@ -14,12 +14,11 @@ CometNativeColumnarToRow : : : : +- CometBroadcastHashJoin : : : : :- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- CometSubqueryBroadcast + : : : : : +- CometBroadcastExchange + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : +- CometBroadcastExchange : : : : +- CometProject : : : : +- CometFilter @@ -49,12 +48,11 @@ CometNativeColumnarToRow : : : : +- CometBroadcastHashJoin : : : : :- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- CometSubqueryBroadcast + : : : : : +- CometBroadcastExchange + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : +- CometBroadcastExchange : : : : +- CometProject : : : : +- CometFilter @@ -84,12 +82,11 @@ CometNativeColumnarToRow : : : +- CometBroadcastHashJoin : : : :- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometSubqueryBroadcast + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter @@ -106,4 +103,4 @@ CometNativeColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.item -Comet accelerated 97 out of 103 eligible operators (94%). Final plan contains 4 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 100 out of 103 eligible operators (97%). Final plan contains 1 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q34.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q34.native_datafusion/extended.txt index 1b61be2937..4c0ef9abcd 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q34.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q34.native_datafusion/extended.txt @@ -15,12 +15,11 @@ CometNativeColumnarToRow : : : +- CometBroadcastHashJoin : : : :- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometSubqueryBroadcast + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter @@ -38,4 +37,4 @@ CometNativeColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.customer -Comet accelerated 35 out of 37 eligible operators (94%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 36 out of 37 eligible operators (97%). Final plan contains 1 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q35.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q35.native_datafusion/extended.txt index 2785c3d0b1..2e79f6f464 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q35.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q35.native_datafusion/extended.txt @@ -19,12 +19,11 @@ TakeOrderedAndProject : : : : +- CometProject : : : : +- CometBroadcastHashJoin : : : : :- CometNativeScan parquet spark_catalog.default.store_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- CometSubqueryBroadcast + : : : : : +- CometBroadcastExchange + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : +- CometBroadcastExchange : : : : +- CometProject : : : : +- CometFilter @@ -34,12 +33,11 @@ TakeOrderedAndProject : : : +- CometProject : : : +- CometBroadcastHashJoin : : : :- CometNativeScan parquet spark_catalog.default.web_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometSubqueryBroadcast + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter @@ -49,12 +47,11 @@ TakeOrderedAndProject : : +- CometProject : : +- CometBroadcastHashJoin : : :- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometSubqueryBroadcast + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- CometBroadcastExchange : : +- CometProject : : +- CometFilter @@ -70,4 +67,4 @@ TakeOrderedAndProject +- CometFilter +- CometNativeScan parquet spark_catalog.default.customer_demographics -Comet accelerated 41 out of 62 eligible operators (66%). Final plan contains 9 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 44 out of 62 eligible operators (70%). Final plan contains 6 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q35a.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q35a.native_datafusion/extended.txt index cba125f412..173c450c71 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q35a.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q35a.native_datafusion/extended.txt @@ -16,12 +16,11 @@ CometNativeColumnarToRow : : : +- CometProject : : : +- CometBroadcastHashJoin : : : :- CometNativeScan parquet spark_catalog.default.store_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometSubqueryBroadcast + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter @@ -31,12 +30,11 @@ CometNativeColumnarToRow : : :- CometProject : : : +- CometBroadcastHashJoin : : : :- CometNativeScan parquet spark_catalog.default.web_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometSubqueryBroadcast + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter @@ -44,12 +42,11 @@ CometNativeColumnarToRow : : +- CometProject : : +- CometBroadcastHashJoin : : :- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometSubqueryBroadcast + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- CometBroadcastExchange : : +- CometProject : : +- CometFilter @@ -63,4 +60,4 @@ CometNativeColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.customer_demographics -Comet accelerated 54 out of 60 eligible operators (90%). Final plan contains 4 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 57 out of 60 eligible operators (95%). Final plan contains 1 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q36a.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q36a.native_datafusion/extended.txt index 0a69bca968..be0d09e04e 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q36a.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q36a.native_datafusion/extended.txt @@ -19,12 +19,11 @@ TakeOrderedAndProject : : : +- CometBroadcastHashJoin : : : :- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometSubqueryBroadcast + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter @@ -51,12 +50,11 @@ TakeOrderedAndProject : : : +- CometBroadcastHashJoin : : : :- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometSubqueryBroadcast + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter @@ -83,12 +81,11 @@ TakeOrderedAndProject : : +- CometBroadcastHashJoin : : :- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometSubqueryBroadcast + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- CometBroadcastExchange : : +- CometProject : : +- CometFilter @@ -102,4 +99,4 @@ TakeOrderedAndProject +- CometFilter +- CometNativeScan parquet spark_catalog.default.store -Comet accelerated 90 out of 99 eligible operators (90%). Final plan contains 4 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 93 out of 99 eligible operators (93%). Final plan contains 1 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q47.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q47.native_datafusion/extended.txt index 0886059452..54fa1daf2e 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q47.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q47.native_datafusion/extended.txt @@ -26,11 +26,10 @@ TakeOrderedAndProject : : : : +- CometBroadcastExchange : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometSubqueryBroadcast + : : : : +- CometBroadcastExchange + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- CometBroadcastExchange : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -58,11 +57,10 @@ TakeOrderedAndProject : : : +- CometBroadcastExchange : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometSubqueryBroadcast + : : : +- CometBroadcastExchange + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- CometBroadcastExchange : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -90,11 +88,10 @@ TakeOrderedAndProject : : +- CometBroadcastExchange : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometSubqueryBroadcast + : : +- CometBroadcastExchange + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim : +- CometBroadcastExchange : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -102,4 +99,4 @@ TakeOrderedAndProject +- CometFilter +- CometNativeScan parquet spark_catalog.default.store -Comet accelerated 75 out of 97 eligible operators (77%). Final plan contains 6 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 78 out of 97 eligible operators (80%). Final plan contains 3 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q49.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q49.native_datafusion/extended.txt index 6d7dadcde9..f2ae44fb86 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q49.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q49.native_datafusion/extended.txt @@ -23,12 +23,11 @@ CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometSubqueryBroadcast + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- CometProject : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.web_returns @@ -55,12 +54,11 @@ CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometSubqueryBroadcast + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- CometProject : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.catalog_returns @@ -87,12 +85,11 @@ CometNativeColumnarToRow : : +- CometProject : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometSubqueryBroadcast + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim : +- CometProject : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.store_returns @@ -101,4 +98,4 @@ CometNativeColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.date_dim -Comet accelerated 72 out of 95 eligible operators (75%). Final plan contains 7 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 75 out of 95 eligible operators (78%). Final plan contains 4 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q51a.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q51a.native_datafusion/extended.txt index d9b9186c1e..a3400c5ecf 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q51a.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q51a.native_datafusion/extended.txt @@ -31,12 +31,11 @@ TakeOrderedAndProject : : : +- CometBroadcastHashJoin : : : :- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometSubqueryBroadcast + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter @@ -54,12 +53,11 @@ TakeOrderedAndProject : : +- CometBroadcastHashJoin : : :- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometSubqueryBroadcast + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- CometBroadcastExchange : : +- CometProject : : +- CometFilter @@ -84,12 +82,11 @@ TakeOrderedAndProject : : +- CometBroadcastHashJoin : : :- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometSubqueryBroadcast + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- CometBroadcastExchange : : +- CometProject : : +- CometFilter @@ -107,12 +104,11 @@ TakeOrderedAndProject : +- CometBroadcastHashJoin : :- CometFilter : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometSubqueryBroadcast + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim : +- CometBroadcastExchange : +- CometProject : +- CometFilter @@ -146,12 +142,11 @@ TakeOrderedAndProject : : +- CometBroadcastHashJoin : : :- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometSubqueryBroadcast + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- CometBroadcastExchange : : +- CometProject : : +- CometFilter @@ -169,12 +164,11 @@ TakeOrderedAndProject : +- CometBroadcastHashJoin : :- CometFilter : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometSubqueryBroadcast + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim : +- CometBroadcastExchange : +- CometProject : +- CometFilter @@ -199,12 +193,11 @@ TakeOrderedAndProject : +- CometBroadcastHashJoin : :- CometFilter : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometSubqueryBroadcast + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim : +- CometBroadcastExchange : +- CometProject : +- CometFilter @@ -222,15 +215,14 @@ TakeOrderedAndProject +- CometBroadcastHashJoin :- CometFilter : +- CometNativeScan parquet spark_catalog.default.store_sales - : +- SubqueryBroadcast - : +- BroadcastExchange - : +- CometNativeColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometSubqueryBroadcast + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.date_dim +- CometBroadcastExchange +- CometProject +- CometFilter +- CometNativeScan parquet spark_catalog.default.date_dim -Comet accelerated 150 out of 212 eligible operators (70%). Final plan contains 22 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 158 out of 212 eligible operators (74%). Final plan contains 14 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q57.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q57.native_datafusion/extended.txt index 024bfeeeea..aa4ccf3246 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q57.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q57.native_datafusion/extended.txt @@ -26,11 +26,10 @@ TakeOrderedAndProject : : : : +- CometBroadcastExchange : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometSubqueryBroadcast + : : : : +- CometBroadcastExchange + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- CometBroadcastExchange : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -58,11 +57,10 @@ TakeOrderedAndProject : : : +- CometBroadcastExchange : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometSubqueryBroadcast + : : : +- CometBroadcastExchange + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- CometBroadcastExchange : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -90,11 +88,10 @@ TakeOrderedAndProject : : +- CometBroadcastExchange : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometSubqueryBroadcast + : : +- CometBroadcastExchange + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim : +- CometBroadcastExchange : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -102,4 +99,4 @@ TakeOrderedAndProject +- CometFilter +- CometNativeScan parquet spark_catalog.default.call_center -Comet accelerated 75 out of 97 eligible operators (77%). Final plan contains 6 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 78 out of 97 eligible operators (80%). Final plan contains 3 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q5a.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q5a.native_datafusion/extended.txt index 9cc06fb361..b472530c10 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q5a.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q5a.native_datafusion/extended.txt @@ -19,21 +19,19 @@ CometNativeColumnarToRow : : : : :- CometProject : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- CometSubqueryBroadcast + : : : : : +- CometBroadcastExchange + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : +- CometProject : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.store_returns - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometSubqueryBroadcast + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter @@ -53,21 +51,19 @@ CometNativeColumnarToRow : : : : :- CometProject : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- CometSubqueryBroadcast + : : : : : +- CometBroadcastExchange + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : +- CometProject : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.catalog_returns - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometSubqueryBroadcast + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter @@ -87,22 +83,20 @@ CometNativeColumnarToRow : : : :- CometProject : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometSubqueryBroadcast + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- CometProject : : : +- CometBroadcastHashJoin : : : :- CometBroadcastExchange : : : : +- CometNativeScan parquet spark_catalog.default.web_returns - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometSubqueryBroadcast + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.web_sales @@ -132,21 +126,19 @@ CometNativeColumnarToRow : : : : :- CometProject : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- CometSubqueryBroadcast + : : : : : +- CometBroadcastExchange + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : +- CometProject : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.store_returns - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometSubqueryBroadcast + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter @@ -166,21 +158,19 @@ CometNativeColumnarToRow : : : : :- CometProject : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- CometSubqueryBroadcast + : : : : : +- CometBroadcastExchange + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : +- CometProject : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.catalog_returns - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometSubqueryBroadcast + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter @@ -200,22 +190,20 @@ CometNativeColumnarToRow : : : :- CometProject : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometSubqueryBroadcast + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- CometProject : : : +- CometBroadcastHashJoin : : : :- CometBroadcastExchange : : : : +- CometNativeScan parquet spark_catalog.default.web_returns - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometSubqueryBroadcast + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.web_sales @@ -245,21 +233,19 @@ CometNativeColumnarToRow : : : :- CometProject : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometSubqueryBroadcast + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.store_returns - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometSubqueryBroadcast + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- CometBroadcastExchange : : +- CometProject : : +- CometFilter @@ -279,21 +265,19 @@ CometNativeColumnarToRow : : : :- CometProject : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometSubqueryBroadcast + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.catalog_returns - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometSubqueryBroadcast + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- CometBroadcastExchange : : +- CometProject : : +- CometFilter @@ -313,22 +297,20 @@ CometNativeColumnarToRow : : :- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometSubqueryBroadcast + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- CometProject : : +- CometBroadcastHashJoin : : :- CometBroadcastExchange : : : +- CometNativeScan parquet spark_catalog.default.web_returns - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometSubqueryBroadcast + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- CometProject : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.web_sales @@ -341,4 +323,4 @@ CometNativeColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.web_site -Comet accelerated 287 out of 323 eligible operators (88%). Final plan contains 19 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 305 out of 323 eligible operators (94%). Final plan contains 1 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q6.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q6.native_datafusion/extended.txt index 1cb21cb650..4b31bf751a 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q6.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q6.native_datafusion/extended.txt @@ -21,24 +21,30 @@ CometNativeColumnarToRow : : +- CometBroadcastExchange : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometProject - : : +- CometFilter - : : : +- Subquery - : : : +- CometNativeColumnarToRow - : : : +- CometHashAggregate - : : : +- CometExchange - : : : +- CometHashAggregate - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometSubqueryBroadcast + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : : +- Subquery + : : : +- CometNativeColumnarToRow + : : : +- CometHashAggregate + : : : +- CometExchange + : : : +- CometHashAggregate + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometNativeScan parquet spark_catalog.default.date_dim : +- CometBroadcastExchange : +- CometProject : +- CometFilter - : : +- ReusedSubquery + : : +- Subquery + : : +- CometNativeColumnarToRow + : : +- CometHashAggregate + : : +- CometExchange + : : +- CometHashAggregate + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim : +- CometNativeScan parquet spark_catalog.default.date_dim +- CometBroadcastExchange +- CometProject @@ -54,4 +60,4 @@ CometNativeColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.item -Comet accelerated 48 out of 52 eligible operators (92%). Final plan contains 3 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 55 out of 58 eligible operators (94%). Final plan contains 3 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q64.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q64.native_datafusion/extended.txt index f1fe6fb3e3..14ffe4eab9 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q64.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q64.native_datafusion/extended.txt @@ -46,11 +46,10 @@ CometNativeColumnarToRow : : : : : : : : : : : : : : : : : :- CometBroadcastExchange : : : : : : : : : : : : : : : : : : +- CometFilter : : : : : : : : : : : : : : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : : : : : : : : : : : : : : : : +- SubqueryBroadcast - : : : : : : : : : : : : : : : : : : +- BroadcastExchange - : : : : : : : : : : : : : : : : : : +- CometNativeColumnarToRow - : : : : : : : : : : : : : : : : : : +- CometFilter - : : : : : : : : : : : : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : : : : : : : : : : : : : +- CometSubqueryBroadcast + : : : : : : : : : : : : : : : : : : +- CometBroadcastExchange + : : : : : : : : : : : : : : : : : : +- CometFilter + : : : : : : : : : : : : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : : : : : : : : : : : : : : +- CometProject : : : : : : : : : : : : : : : : : +- CometFilter : : : : : : : : : : : : : : : : : +- CometNativeScan parquet spark_catalog.default.store_returns @@ -166,11 +165,10 @@ CometNativeColumnarToRow : : : : : : : : : : : : : : : : :- CometBroadcastExchange : : : : : : : : : : : : : : : : : +- CometFilter : : : : : : : : : : : : : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : : : : : : : : : : : : : : : +- SubqueryBroadcast - : : : : : : : : : : : : : : : : : +- BroadcastExchange - : : : : : : : : : : : : : : : : : +- CometNativeColumnarToRow - : : : : : : : : : : : : : : : : : +- CometFilter - : : : : : : : : : : : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : : : : : : : : : : : : +- CometSubqueryBroadcast + : : : : : : : : : : : : : : : : : +- CometBroadcastExchange + : : : : : : : : : : : : : : : : : +- CometFilter + : : : : : : : : : : : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : : : : : : : : : : : : : +- CometProject : : : : : : : : : : : : : : : : +- CometFilter : : : : : : : : : : : : : : : : +- CometNativeScan parquet spark_catalog.default.store_returns @@ -244,4 +242,4 @@ CometNativeColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.item -Comet accelerated 238 out of 242 eligible operators (98%). Final plan contains 3 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 240 out of 242 eligible operators (99%). Final plan contains 1 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q67a.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q67a.native_datafusion/extended.txt index abf8dad401..8abd1fc15f 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q67a.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q67a.native_datafusion/extended.txt @@ -20,12 +20,11 @@ TakeOrderedAndProject : : : +- CometBroadcastHashJoin : : : :- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometSubqueryBroadcast + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter @@ -52,12 +51,11 @@ TakeOrderedAndProject : : : +- CometBroadcastHashJoin : : : :- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometSubqueryBroadcast + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter @@ -84,12 +82,11 @@ TakeOrderedAndProject : : : +- CometBroadcastHashJoin : : : :- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometSubqueryBroadcast + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter @@ -116,12 +113,11 @@ TakeOrderedAndProject : : : +- CometBroadcastHashJoin : : : :- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometSubqueryBroadcast + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter @@ -148,12 +144,11 @@ TakeOrderedAndProject : : : +- CometBroadcastHashJoin : : : :- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometSubqueryBroadcast + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter @@ -180,12 +175,11 @@ TakeOrderedAndProject : : : +- CometBroadcastHashJoin : : : :- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometSubqueryBroadcast + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter @@ -212,12 +206,11 @@ TakeOrderedAndProject : : : +- CometBroadcastHashJoin : : : :- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometSubqueryBroadcast + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter @@ -244,12 +237,11 @@ TakeOrderedAndProject : : : +- CometBroadcastHashJoin : : : :- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometSubqueryBroadcast + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter @@ -276,12 +268,11 @@ TakeOrderedAndProject : : +- CometBroadcastHashJoin : : :- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometSubqueryBroadcast + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- CometBroadcastExchange : : +- CometProject : : +- CometFilter @@ -295,4 +286,4 @@ TakeOrderedAndProject +- CometFilter +- CometNativeScan parquet spark_catalog.default.item -Comet accelerated 262 out of 285 eligible operators (91%). Final plan contains 11 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 271 out of 285 eligible operators (95%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q70a.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q70a.native_datafusion/extended.txt index 4ba9e6eb00..97dd294f61 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q70a.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q70a.native_datafusion/extended.txt @@ -19,12 +19,11 @@ TakeOrderedAndProject : : +- CometBroadcastHashJoin : : :- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometSubqueryBroadcast + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- CometBroadcastExchange : : +- CometProject : : +- CometFilter @@ -51,12 +50,11 @@ TakeOrderedAndProject : : +- CometBroadcastHashJoin : : :- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometSubqueryBroadcast + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- CometBroadcastExchange : : +- CometProject : : +- CometFilter @@ -80,12 +78,11 @@ TakeOrderedAndProject : : +- CometBroadcastHashJoin : : :- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometSubqueryBroadcast + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- CometBroadcastExchange : : +- CometProject : : +- CometFilter @@ -112,12 +109,11 @@ TakeOrderedAndProject : : +- CometBroadcastHashJoin : : :- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometSubqueryBroadcast + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- CometBroadcastExchange : : +- CometProject : : +- CometFilter @@ -141,12 +137,11 @@ TakeOrderedAndProject : +- CometBroadcastHashJoin : :- CometFilter : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometSubqueryBroadcast + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim : +- CometBroadcastExchange : +- CometProject : +- CometFilter @@ -173,12 +168,11 @@ TakeOrderedAndProject : +- CometBroadcastHashJoin : :- CometFilter : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometSubqueryBroadcast + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim : +- CometBroadcastExchange : +- CometProject : +- CometFilter @@ -188,4 +182,4 @@ TakeOrderedAndProject +- CometFilter +- CometNativeScan parquet spark_catalog.default.date_dim -Comet accelerated 111 out of 168 eligible operators (66%). Final plan contains 21 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 117 out of 168 eligible operators (69%). Final plan contains 15 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q72.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q72.native_datafusion/extended.txt index 623a68ddab..d443b31d1e 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q72.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q72.native_datafusion/extended.txt @@ -27,12 +27,11 @@ CometNativeColumnarToRow : : : : : : : : : +- CometBroadcastHashJoin : : : : : : : : : :- CometFilter : : : : : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : : : : : : : +- SubqueryBroadcast - : : : : : : : : : : +- BroadcastExchange - : : : : : : : : : : +- CometNativeColumnarToRow - : : : : : : : : : : +- CometProject - : : : : : : : : : : +- CometFilter - : : : : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : : : : : +- CometSubqueryBroadcast + : : : : : : : : : : +- CometBroadcastExchange + : : : : : : : : : : +- CometProject + : : : : : : : : : : +- CometFilter + : : : : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : : : : : : +- CometBroadcastExchange : : : : : : : : : +- CometFilter : : : : : : : : : +- CometNativeScan parquet spark_catalog.default.inventory @@ -69,4 +68,4 @@ CometNativeColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.catalog_returns -Comet accelerated 66 out of 68 eligible operators (97%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 67 out of 68 eligible operators (98%). Final plan contains 1 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q74.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q74.native_datafusion/extended.txt index 51a3f5ca94..7768138645 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q74.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q74.native_datafusion/extended.txt @@ -19,11 +19,10 @@ CometNativeColumnarToRow : : : : +- CometBroadcastExchange : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometSubqueryBroadcast + : : : : +- CometBroadcastExchange + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- CometBroadcastExchange : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -41,11 +40,10 @@ CometNativeColumnarToRow : : : +- CometBroadcastExchange : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometSubqueryBroadcast + : : : +- CometBroadcastExchange + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- CometBroadcastExchange : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -64,11 +62,10 @@ CometNativeColumnarToRow : : +- CometBroadcastExchange : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometSubqueryBroadcast + : : +- CometBroadcastExchange + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim : +- CometBroadcastExchange : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -86,13 +83,12 @@ CometNativeColumnarToRow : +- CometBroadcastExchange : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.web_sales - : +- SubqueryBroadcast - : +- BroadcastExchange - : +- CometNativeColumnarToRow - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometSubqueryBroadcast + : +- CometBroadcastExchange + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.date_dim +- CometBroadcastExchange +- CometFilter +- CometNativeScan parquet spark_catalog.default.date_dim -Comet accelerated 83 out of 91 eligible operators (91%). Final plan contains 5 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 87 out of 91 eligible operators (95%). Final plan contains 1 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q75.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q75.native_datafusion/extended.txt index a943308f1e..ee7c0af216 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q75.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q75.native_datafusion/extended.txt @@ -22,11 +22,10 @@ CometNativeColumnarToRow : : : : +- CometBroadcastHashJoin : : : : :- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- CometSubqueryBroadcast + : : : : : +- CometBroadcastExchange + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : +- CometBroadcastExchange : : : : +- CometProject : : : : +- CometFilter @@ -49,11 +48,10 @@ CometNativeColumnarToRow : : : : +- CometBroadcastHashJoin : : : : :- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- CometSubqueryBroadcast + : : : : : +- CometBroadcastExchange + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : +- CometBroadcastExchange : : : : +- CometProject : : : : +- CometFilter @@ -76,11 +74,10 @@ CometNativeColumnarToRow : : : +- CometBroadcastHashJoin : : : :- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometSubqueryBroadcast + : : : : +- CometBroadcastExchange + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter @@ -113,11 +110,10 @@ CometNativeColumnarToRow : : : +- CometBroadcastHashJoin : : : :- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometSubqueryBroadcast + : : : : +- CometBroadcastExchange + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter @@ -140,11 +136,10 @@ CometNativeColumnarToRow : : : +- CometBroadcastHashJoin : : : :- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometSubqueryBroadcast + : : : : +- CometBroadcastExchange + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter @@ -167,11 +162,10 @@ CometNativeColumnarToRow : : +- CometBroadcastHashJoin : : :- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometSubqueryBroadcast + : : : +- CometBroadcastExchange + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- CometBroadcastExchange : : +- CometProject : : +- CometFilter @@ -185,4 +179,4 @@ CometNativeColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.web_returns -Comet accelerated 167 out of 179 eligible operators (93%). Final plan contains 7 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 173 out of 179 eligible operators (96%). Final plan contains 1 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q77a.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q77a.native_datafusion/extended.txt index 24407fc0a2..6621cfd8f8 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q77a.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q77a.native_datafusion/extended.txt @@ -21,12 +21,11 @@ CometNativeColumnarToRow : : : : +- CometBroadcastHashJoin : : : : :- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- CometSubqueryBroadcast + : : : : : +- CometBroadcastExchange + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : +- CometBroadcastExchange : : : : +- CometProject : : : : +- CometFilter @@ -44,12 +43,11 @@ CometNativeColumnarToRow : : : +- CometBroadcastHashJoin : : : :- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.store_returns - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometSubqueryBroadcast + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter @@ -67,12 +65,11 @@ CometNativeColumnarToRow : : : +- CometProject : : : +- CometBroadcastHashJoin : : : :- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometSubqueryBroadcast + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter @@ -84,12 +81,11 @@ CometNativeColumnarToRow : : +- CometProject : : +- CometBroadcastHashJoin : : :- CometNativeScan parquet spark_catalog.default.catalog_returns - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometSubqueryBroadcast + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- CometBroadcastExchange : : +- CometProject : : +- CometFilter @@ -106,12 +102,11 @@ CometNativeColumnarToRow : : : +- CometBroadcastHashJoin : : : :- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometSubqueryBroadcast + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter @@ -129,12 +124,11 @@ CometNativeColumnarToRow : : +- CometBroadcastHashJoin : : :- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.web_returns - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometSubqueryBroadcast + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- CometBroadcastExchange : : +- CometProject : : +- CometFilter @@ -163,12 +157,11 @@ CometNativeColumnarToRow : : : : +- CometBroadcastHashJoin : : : : :- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- CometSubqueryBroadcast + : : : : : +- CometBroadcastExchange + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : +- CometBroadcastExchange : : : : +- CometProject : : : : +- CometFilter @@ -186,12 +179,11 @@ CometNativeColumnarToRow : : : +- CometBroadcastHashJoin : : : :- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.store_returns - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometSubqueryBroadcast + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter @@ -209,12 +201,11 @@ CometNativeColumnarToRow : : : +- CometProject : : : +- CometBroadcastHashJoin : : : :- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometSubqueryBroadcast + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter @@ -226,12 +217,11 @@ CometNativeColumnarToRow : : +- CometProject : : +- CometBroadcastHashJoin : : :- CometNativeScan parquet spark_catalog.default.catalog_returns - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometSubqueryBroadcast + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- CometBroadcastExchange : : +- CometProject : : +- CometFilter @@ -248,12 +238,11 @@ CometNativeColumnarToRow : : : +- CometBroadcastHashJoin : : : :- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometSubqueryBroadcast + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter @@ -271,12 +260,11 @@ CometNativeColumnarToRow : : +- CometBroadcastHashJoin : : :- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.web_returns - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometSubqueryBroadcast + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- CometBroadcastExchange : : +- CometProject : : +- CometFilter @@ -305,12 +293,11 @@ CometNativeColumnarToRow : : : +- CometBroadcastHashJoin : : : :- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometSubqueryBroadcast + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter @@ -328,12 +315,11 @@ CometNativeColumnarToRow : : +- CometBroadcastHashJoin : : :- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.store_returns - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometSubqueryBroadcast + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- CometBroadcastExchange : : +- CometProject : : +- CometFilter @@ -351,12 +337,11 @@ CometNativeColumnarToRow : : +- CometProject : : +- CometBroadcastHashJoin : : :- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometSubqueryBroadcast + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- CometBroadcastExchange : : +- CometProject : : +- CometFilter @@ -368,12 +353,11 @@ CometNativeColumnarToRow : +- CometProject : +- CometBroadcastHashJoin : :- CometNativeScan parquet spark_catalog.default.catalog_returns - : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometSubqueryBroadcast + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim : +- CometBroadcastExchange : +- CometProject : +- CometFilter @@ -390,12 +374,11 @@ CometNativeColumnarToRow : : +- CometBroadcastHashJoin : : :- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometSubqueryBroadcast + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- CometBroadcastExchange : : +- CometProject : : +- CometFilter @@ -413,12 +396,11 @@ CometNativeColumnarToRow : +- CometBroadcastHashJoin : :- CometFilter : : +- CometNativeScan parquet spark_catalog.default.web_returns - : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometSubqueryBroadcast + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim : +- CometBroadcastExchange : +- CometProject : +- CometFilter @@ -427,4 +409,4 @@ CometNativeColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.web_page -Comet accelerated 332 out of 392 eligible operators (84%). Final plan contains 36 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 350 out of 392 eligible operators (89%). Final plan contains 18 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q78.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q78.native_datafusion/extended.txt index ebfdcc3c62..8426b9653c 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q78.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q78.native_datafusion/extended.txt @@ -17,11 +17,10 @@ TakeOrderedAndProject : : : : +- CometExchange : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometSubqueryBroadcast + : : : : +- CometBroadcastExchange + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- CometSort : : : +- CometExchange : : : +- CometProject @@ -44,11 +43,10 @@ TakeOrderedAndProject : : : +- CometExchange : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometSubqueryBroadcast + : : : +- CometBroadcastExchange + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- CometSort : : +- CometExchange : : +- CometProject @@ -71,11 +69,10 @@ TakeOrderedAndProject : : +- CometExchange : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometSubqueryBroadcast + : : +- CometBroadcastExchange + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim : +- CometSort : +- CometExchange : +- CometProject @@ -85,4 +82,4 @@ TakeOrderedAndProject +- CometFilter +- CometNativeScan parquet spark_catalog.default.date_dim -Comet accelerated 74 out of 82 eligible operators (90%). Final plan contains 4 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 77 out of 82 eligible operators (93%). Final plan contains 1 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q80a.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q80a.native_datafusion/extended.txt index ec566b42de..dced05333a 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q80a.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q80a.native_datafusion/extended.txt @@ -25,12 +25,11 @@ CometNativeColumnarToRow : : : : : : : +- CometExchange : : : : : : : +- CometFilter : : : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : : : : : +- SubqueryBroadcast - : : : : : : : +- BroadcastExchange - : : : : : : : +- CometNativeColumnarToRow - : : : : : : : +- CometProject - : : : : : : : +- CometFilter - : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : : +- CometSubqueryBroadcast + : : : : : : : +- CometBroadcastExchange + : : : : : : : +- CometProject + : : : : : : : +- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : : : +- CometSort : : : : : : +- CometExchange : : : : : : +- CometProject @@ -69,12 +68,11 @@ CometNativeColumnarToRow : : : : : : : +- CometExchange : : : : : : : +- CometFilter : : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : : : : +- SubqueryBroadcast - : : : : : : : +- BroadcastExchange - : : : : : : : +- CometNativeColumnarToRow - : : : : : : : +- CometProject - : : : : : : : +- CometFilter - : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : : +- CometSubqueryBroadcast + : : : : : : : +- CometBroadcastExchange + : : : : : : : +- CometProject + : : : : : : : +- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : : : +- CometSort : : : : : : +- CometExchange : : : : : : +- CometProject @@ -113,12 +111,11 @@ CometNativeColumnarToRow : : : : : : +- CometExchange : : : : : : +- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometNativeColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : +- CometSubqueryBroadcast + : : : : : : +- CometBroadcastExchange + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : : +- CometSort : : : : : +- CometExchange : : : : : +- CometProject @@ -164,12 +161,11 @@ CometNativeColumnarToRow : : : : : : : +- CometExchange : : : : : : : +- CometFilter : : : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : : : : : +- SubqueryBroadcast - : : : : : : : +- BroadcastExchange - : : : : : : : +- CometNativeColumnarToRow - : : : : : : : +- CometProject - : : : : : : : +- CometFilter - : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : : +- CometSubqueryBroadcast + : : : : : : : +- CometBroadcastExchange + : : : : : : : +- CometProject + : : : : : : : +- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : : : +- CometSort : : : : : : +- CometExchange : : : : : : +- CometProject @@ -208,12 +204,11 @@ CometNativeColumnarToRow : : : : : : : +- CometExchange : : : : : : : +- CometFilter : : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : : : : +- SubqueryBroadcast - : : : : : : : +- BroadcastExchange - : : : : : : : +- CometNativeColumnarToRow - : : : : : : : +- CometProject - : : : : : : : +- CometFilter - : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : : +- CometSubqueryBroadcast + : : : : : : : +- CometBroadcastExchange + : : : : : : : +- CometProject + : : : : : : : +- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : : : +- CometSort : : : : : : +- CometExchange : : : : : : +- CometProject @@ -252,12 +247,11 @@ CometNativeColumnarToRow : : : : : : +- CometExchange : : : : : : +- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometNativeColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : +- CometSubqueryBroadcast + : : : : : : +- CometBroadcastExchange + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : : +- CometSort : : : : : +- CometExchange : : : : : +- CometProject @@ -303,12 +297,11 @@ CometNativeColumnarToRow : : : : : : +- CometExchange : : : : : : +- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometNativeColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : +- CometSubqueryBroadcast + : : : : : : +- CometBroadcastExchange + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : : +- CometSort : : : : : +- CometExchange : : : : : +- CometProject @@ -347,12 +340,11 @@ CometNativeColumnarToRow : : : : : : +- CometExchange : : : : : : +- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometNativeColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : +- CometSubqueryBroadcast + : : : : : : +- CometBroadcastExchange + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : : +- CometSort : : : : : +- CometExchange : : : : : +- CometProject @@ -391,12 +383,11 @@ CometNativeColumnarToRow : : : : : +- CometExchange : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- CometSubqueryBroadcast + : : : : : +- CometBroadcastExchange + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : +- CometSort : : : : +- CometExchange : : : : +- CometProject @@ -419,4 +410,4 @@ CometNativeColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.promotion -Comet accelerated 392 out of 410 eligible operators (95%). Final plan contains 10 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 401 out of 410 eligible operators (97%). Final plan contains 1 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q86a.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q86a.native_datafusion/extended.txt index c1df720935..4c66209a5e 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q86a.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q86a.native_datafusion/extended.txt @@ -17,12 +17,11 @@ TakeOrderedAndProject : : +- CometBroadcastHashJoin : : :- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometSubqueryBroadcast + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- CometBroadcastExchange : : +- CometProject : : +- CometFilter @@ -43,12 +42,11 @@ TakeOrderedAndProject : : +- CometBroadcastHashJoin : : :- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometSubqueryBroadcast + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- CometBroadcastExchange : : +- CometProject : : +- CometFilter @@ -69,12 +67,11 @@ TakeOrderedAndProject : +- CometBroadcastHashJoin : :- CometFilter : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometSubqueryBroadcast + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim : +- CometBroadcastExchange : +- CometProject : +- CometFilter @@ -84,4 +81,4 @@ TakeOrderedAndProject +- CometFilter +- CometNativeScan parquet spark_catalog.default.item -Comet accelerated 72 out of 81 eligible operators (88%). Final plan contains 4 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 75 out of 81 eligible operators (92%). Final plan contains 1 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q98.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q98.native_datafusion/extended.txt index 6c2b043008..ccf9f25336 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q98.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q98.native_datafusion/extended.txt @@ -15,12 +15,11 @@ CometNativeColumnarToRow : +- CometBroadcastHashJoin : :- CometFilter : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometSubqueryBroadcast + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim : +- CometBroadcastExchange : +- CometProject : +- CometFilter @@ -30,4 +29,4 @@ CometNativeColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.date_dim -Comet accelerated 24 out of 28 eligible operators (85%). Final plan contains 3 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 25 out of 28 eligible operators (89%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q10a.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q10a.native_datafusion/extended.txt index cba125f412..173c450c71 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q10a.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q10a.native_datafusion/extended.txt @@ -16,12 +16,11 @@ CometNativeColumnarToRow : : : +- CometProject : : : +- CometBroadcastHashJoin : : : :- CometNativeScan parquet spark_catalog.default.store_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometSubqueryBroadcast + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter @@ -31,12 +30,11 @@ CometNativeColumnarToRow : : :- CometProject : : : +- CometBroadcastHashJoin : : : :- CometNativeScan parquet spark_catalog.default.web_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometSubqueryBroadcast + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter @@ -44,12 +42,11 @@ CometNativeColumnarToRow : : +- CometProject : : +- CometBroadcastHashJoin : : :- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometSubqueryBroadcast + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- CometBroadcastExchange : : +- CometProject : : +- CometFilter @@ -63,4 +60,4 @@ CometNativeColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.customer_demographics -Comet accelerated 54 out of 60 eligible operators (90%). Final plan contains 4 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 57 out of 60 eligible operators (95%). Final plan contains 1 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q11.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q11.native_datafusion/extended.txt index 51a3f5ca94..7768138645 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q11.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q11.native_datafusion/extended.txt @@ -19,11 +19,10 @@ CometNativeColumnarToRow : : : : +- CometBroadcastExchange : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometSubqueryBroadcast + : : : : +- CometBroadcastExchange + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- CometBroadcastExchange : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -41,11 +40,10 @@ CometNativeColumnarToRow : : : +- CometBroadcastExchange : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometSubqueryBroadcast + : : : +- CometBroadcastExchange + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- CometBroadcastExchange : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -64,11 +62,10 @@ CometNativeColumnarToRow : : +- CometBroadcastExchange : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometSubqueryBroadcast + : : +- CometBroadcastExchange + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim : +- CometBroadcastExchange : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -86,13 +83,12 @@ CometNativeColumnarToRow : +- CometBroadcastExchange : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.web_sales - : +- SubqueryBroadcast - : +- BroadcastExchange - : +- CometNativeColumnarToRow - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometSubqueryBroadcast + : +- CometBroadcastExchange + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.date_dim +- CometBroadcastExchange +- CometFilter +- CometNativeScan parquet spark_catalog.default.date_dim -Comet accelerated 83 out of 91 eligible operators (91%). Final plan contains 5 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 87 out of 91 eligible operators (95%). Final plan contains 1 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q12.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q12.native_datafusion/extended.txt index a8c2531a38..a6cc526735 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q12.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q12.native_datafusion/extended.txt @@ -13,12 +13,11 @@ TakeOrderedAndProject : +- CometBroadcastHashJoin : :- CometFilter : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometSubqueryBroadcast + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim : +- CometBroadcastExchange : +- CometProject : +- CometFilter @@ -28,4 +27,4 @@ TakeOrderedAndProject +- CometFilter +- CometNativeScan parquet spark_catalog.default.date_dim -Comet accelerated 22 out of 27 eligible operators (81%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 23 out of 27 eligible operators (85%). Final plan contains 1 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q14.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q14.native_datafusion/extended.txt index b48995c0a5..341ae1fb04 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q14.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q14.native_datafusion/extended.txt @@ -11,12 +11,11 @@ CometNativeColumnarToRow : : :- CometProject : : : +- CometBroadcastHashJoin : : : :- CometNativeScan parquet spark_catalog.default.store_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometSubqueryBroadcast + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter @@ -24,12 +23,11 @@ CometNativeColumnarToRow : : :- CometProject : : : +- CometBroadcastHashJoin : : : :- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometSubqueryBroadcast + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter @@ -37,12 +35,11 @@ CometNativeColumnarToRow : : +- CometProject : : +- CometBroadcastHashJoin : : :- CometNativeScan parquet spark_catalog.default.web_sales - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometSubqueryBroadcast + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- CometBroadcastExchange : : +- CometProject : : +- CometFilter @@ -57,22 +54,21 @@ CometNativeColumnarToRow : : :- CometBroadcastHashJoin : : : :- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : : +- Subquery - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : +- Subquery - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometSubqueryBroadcast + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : : +- Subquery + : : : : : +- CometNativeColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- Subquery + : : : : +- CometNativeColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometBroadcastHashJoin @@ -89,12 +85,11 @@ CometNativeColumnarToRow : : : : : +- CometBroadcastHashJoin : : : : : :- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometNativeColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : +- CometSubqueryBroadcast + : : : : : : +- CometBroadcastExchange + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : : +- CometBroadcastExchange : : : : : +- CometBroadcastHashJoin : : : : : :- CometFilter @@ -106,12 +101,11 @@ CometNativeColumnarToRow : : : : : : +- CometBroadcastHashJoin : : : : : : :- CometFilter : : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : : : : +- SubqueryBroadcast - : : : : : : : +- BroadcastExchange - : : : : : : : +- CometNativeColumnarToRow - : : : : : : : +- CometProject - : : : : : : : +- CometFilter - : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : : +- CometSubqueryBroadcast + : : : : : : : +- CometBroadcastExchange + : : : : : : : +- CometProject + : : : : : : : +- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : : : +- CometBroadcastExchange : : : : : : +- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.item @@ -130,12 +124,11 @@ CometNativeColumnarToRow : : : : +- CometBroadcastHashJoin : : : : :- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- CometSubqueryBroadcast + : : : : : +- CometBroadcastExchange + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : +- CometBroadcastExchange : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.item @@ -163,12 +156,11 @@ CometNativeColumnarToRow : : : : +- CometBroadcastHashJoin : : : : :- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- CometSubqueryBroadcast + : : : : : +- CometBroadcastExchange + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : +- CometBroadcastExchange : : : : +- CometBroadcastHashJoin : : : : :- CometFilter @@ -180,12 +172,11 @@ CometNativeColumnarToRow : : : : : +- CometBroadcastHashJoin : : : : : :- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometNativeColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : +- CometSubqueryBroadcast + : : : : : : +- CometBroadcastExchange + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : : +- CometBroadcastExchange : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.item @@ -204,12 +195,11 @@ CometNativeColumnarToRow : : : +- CometBroadcastHashJoin : : : :- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometSubqueryBroadcast + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- CometBroadcastExchange : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.item @@ -240,22 +230,21 @@ CometNativeColumnarToRow : :- CometBroadcastHashJoin : : :- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : : +- Subquery - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- Subquery - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometSubqueryBroadcast + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : : +- Subquery + : : : : +- CometNativeColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- Subquery + : : : +- CometNativeColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- CometBroadcastExchange : : +- CometProject : : +- CometBroadcastHashJoin @@ -272,12 +261,11 @@ CometNativeColumnarToRow : : : : +- CometBroadcastHashJoin : : : : :- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- CometSubqueryBroadcast + : : : : : +- CometBroadcastExchange + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : +- CometBroadcastExchange : : : : +- CometBroadcastHashJoin : : : : :- CometFilter @@ -289,12 +277,11 @@ CometNativeColumnarToRow : : : : : +- CometBroadcastHashJoin : : : : : :- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometNativeColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : +- CometSubqueryBroadcast + : : : : : : +- CometBroadcastExchange + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : : +- CometBroadcastExchange : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.item @@ -313,12 +300,11 @@ CometNativeColumnarToRow : : : +- CometBroadcastHashJoin : : : :- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometSubqueryBroadcast + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- CometBroadcastExchange : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.item @@ -346,12 +332,11 @@ CometNativeColumnarToRow : : : +- CometBroadcastHashJoin : : : :- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometSubqueryBroadcast + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- CometBroadcastExchange : : : +- CometBroadcastHashJoin : : : :- CometFilter @@ -363,12 +348,11 @@ CometNativeColumnarToRow : : : : +- CometBroadcastHashJoin : : : : :- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- CometSubqueryBroadcast + : : : : : +- CometBroadcastExchange + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : +- CometBroadcastExchange : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.item @@ -387,12 +371,11 @@ CometNativeColumnarToRow : : +- CometBroadcastHashJoin : : :- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometSubqueryBroadcast + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- CometBroadcastExchange : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.item @@ -411,4 +394,4 @@ CometNativeColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.date_dim -Comet accelerated 343 out of 387 eligible operators (88%). Final plan contains 25 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 360 out of 387 eligible operators (93%). Final plan contains 8 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q14a.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q14a.native_datafusion/extended.txt index 636e5ce377..83b545c34c 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q14a.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q14a.native_datafusion/extended.txt @@ -18,12 +18,11 @@ CometNativeColumnarToRow : : : :- CometProject : : : : +- CometBroadcastHashJoin : : : : :- CometNativeScan parquet spark_catalog.default.store_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- CometSubqueryBroadcast + : : : : : +- CometBroadcastExchange + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : +- CometBroadcastExchange : : : : +- CometProject : : : : +- CometFilter @@ -31,12 +30,11 @@ CometNativeColumnarToRow : : : :- CometProject : : : : +- CometBroadcastHashJoin : : : : :- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- CometSubqueryBroadcast + : : : : : +- CometBroadcastExchange + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : +- CometBroadcastExchange : : : : +- CometProject : : : : +- CometFilter @@ -44,12 +42,11 @@ CometNativeColumnarToRow : : : +- CometProject : : : +- CometBroadcastHashJoin : : : :- CometNativeScan parquet spark_catalog.default.web_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometSubqueryBroadcast + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter @@ -64,12 +61,11 @@ CometNativeColumnarToRow : : : :- CometBroadcastHashJoin : : : : :- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- CometSubqueryBroadcast + : : : : : +- CometBroadcastExchange + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : +- CometBroadcastExchange : : : : +- CometProject : : : : +- CometBroadcastHashJoin @@ -86,12 +82,11 @@ CometNativeColumnarToRow : : : : : : +- CometBroadcastHashJoin : : : : : : :- CometFilter : : : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : : : : : +- SubqueryBroadcast - : : : : : : : +- BroadcastExchange - : : : : : : : +- CometNativeColumnarToRow - : : : : : : : +- CometProject - : : : : : : : +- CometFilter - : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : : +- CometSubqueryBroadcast + : : : : : : : +- CometBroadcastExchange + : : : : : : : +- CometProject + : : : : : : : +- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : : : +- CometBroadcastExchange : : : : : : +- CometBroadcastHashJoin : : : : : : :- CometFilter @@ -103,12 +98,11 @@ CometNativeColumnarToRow : : : : : : : +- CometBroadcastHashJoin : : : : : : : :- CometFilter : : : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : : : : : +- SubqueryBroadcast - : : : : : : : : +- BroadcastExchange - : : : : : : : : +- CometNativeColumnarToRow - : : : : : : : : +- CometProject - : : : : : : : : +- CometFilter - : : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : : : +- CometSubqueryBroadcast + : : : : : : : : +- CometBroadcastExchange + : : : : : : : : +- CometProject + : : : : : : : : +- CometFilter + : : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : : : : +- CometBroadcastExchange : : : : : : : +- CometFilter : : : : : : : +- CometNativeScan parquet spark_catalog.default.item @@ -127,12 +121,11 @@ CometNativeColumnarToRow : : : : : +- CometBroadcastHashJoin : : : : : :- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometNativeColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : +- CometSubqueryBroadcast + : : : : : : +- CometBroadcastExchange + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : : +- CometBroadcastExchange : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.item @@ -160,12 +153,11 @@ CometNativeColumnarToRow : : : : : +- CometBroadcastHashJoin : : : : : :- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometNativeColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : +- CometSubqueryBroadcast + : : : : : : +- CometBroadcastExchange + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : : +- CometBroadcastExchange : : : : : +- CometBroadcastHashJoin : : : : : :- CometFilter @@ -177,12 +169,11 @@ CometNativeColumnarToRow : : : : : : +- CometBroadcastHashJoin : : : : : : :- CometFilter : : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : : : : +- SubqueryBroadcast - : : : : : : : +- BroadcastExchange - : : : : : : : +- CometNativeColumnarToRow - : : : : : : : +- CometProject - : : : : : : : +- CometFilter - : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : : +- CometSubqueryBroadcast + : : : : : : : +- CometBroadcastExchange + : : : : : : : +- CometProject + : : : : : : : +- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : : : +- CometBroadcastExchange : : : : : : +- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.item @@ -201,12 +192,11 @@ CometNativeColumnarToRow : : : : +- CometBroadcastHashJoin : : : : :- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- CometSubqueryBroadcast + : : : : : +- CometBroadcastExchange + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : +- CometBroadcastExchange : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.item @@ -230,12 +220,11 @@ CometNativeColumnarToRow : : : :- CometBroadcastHashJoin : : : : :- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- CometSubqueryBroadcast + : : : : : +- CometBroadcastExchange + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : +- CometBroadcastExchange : : : : +- CometProject : : : : +- CometBroadcastHashJoin @@ -252,12 +241,11 @@ CometNativeColumnarToRow : : : : : : +- CometBroadcastHashJoin : : : : : : :- CometFilter : : : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : : : : : +- SubqueryBroadcast - : : : : : : : +- BroadcastExchange - : : : : : : : +- CometNativeColumnarToRow - : : : : : : : +- CometProject - : : : : : : : +- CometFilter - : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : : +- CometSubqueryBroadcast + : : : : : : : +- CometBroadcastExchange + : : : : : : : +- CometProject + : : : : : : : +- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : : : +- CometBroadcastExchange : : : : : : +- CometBroadcastHashJoin : : : : : : :- CometFilter @@ -269,12 +257,11 @@ CometNativeColumnarToRow : : : : : : : +- CometBroadcastHashJoin : : : : : : : :- CometFilter : : : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : : : : : +- SubqueryBroadcast - : : : : : : : : +- BroadcastExchange - : : : : : : : : +- CometNativeColumnarToRow - : : : : : : : : +- CometProject - : : : : : : : : +- CometFilter - : : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : : : +- CometSubqueryBroadcast + : : : : : : : : +- CometBroadcastExchange + : : : : : : : : +- CometProject + : : : : : : : : +- CometFilter + : : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : : : : +- CometBroadcastExchange : : : : : : : +- CometFilter : : : : : : : +- CometNativeScan parquet spark_catalog.default.item @@ -293,12 +280,11 @@ CometNativeColumnarToRow : : : : : +- CometBroadcastHashJoin : : : : : :- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometNativeColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : +- CometSubqueryBroadcast + : : : : : : +- CometBroadcastExchange + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : : +- CometBroadcastExchange : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.item @@ -326,12 +312,11 @@ CometNativeColumnarToRow : : : : : +- CometBroadcastHashJoin : : : : : :- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometNativeColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : +- CometSubqueryBroadcast + : : : : : : +- CometBroadcastExchange + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : : +- CometBroadcastExchange : : : : : +- CometBroadcastHashJoin : : : : : :- CometFilter @@ -343,12 +328,11 @@ CometNativeColumnarToRow : : : : : : +- CometBroadcastHashJoin : : : : : : :- CometFilter : : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : : : : +- SubqueryBroadcast - : : : : : : : +- BroadcastExchange - : : : : : : : +- CometNativeColumnarToRow - : : : : : : : +- CometProject - : : : : : : : +- CometFilter - : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : : +- CometSubqueryBroadcast + : : : : : : : +- CometBroadcastExchange + : : : : : : : +- CometProject + : : : : : : : +- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : : : +- CometBroadcastExchange : : : : : : +- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.item @@ -367,12 +351,11 @@ CometNativeColumnarToRow : : : : +- CometBroadcastHashJoin : : : : :- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- CometSubqueryBroadcast + : : : : : +- CometBroadcastExchange + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : +- CometBroadcastExchange : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.item @@ -396,12 +379,11 @@ CometNativeColumnarToRow : : :- CometBroadcastHashJoin : : : :- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometSubqueryBroadcast + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometBroadcastHashJoin @@ -418,12 +400,11 @@ CometNativeColumnarToRow : : : : : +- CometBroadcastHashJoin : : : : : :- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometNativeColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : +- CometSubqueryBroadcast + : : : : : : +- CometBroadcastExchange + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : : +- CometBroadcastExchange : : : : : +- CometBroadcastHashJoin : : : : : :- CometFilter @@ -435,12 +416,11 @@ CometNativeColumnarToRow : : : : : : +- CometBroadcastHashJoin : : : : : : :- CometFilter : : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : : : : +- SubqueryBroadcast - : : : : : : : +- BroadcastExchange - : : : : : : : +- CometNativeColumnarToRow - : : : : : : : +- CometProject - : : : : : : : +- CometFilter - : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : : +- CometSubqueryBroadcast + : : : : : : : +- CometBroadcastExchange + : : : : : : : +- CometProject + : : : : : : : +- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : : : +- CometBroadcastExchange : : : : : : +- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.item @@ -459,12 +439,11 @@ CometNativeColumnarToRow : : : : +- CometBroadcastHashJoin : : : : :- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- CometSubqueryBroadcast + : : : : : +- CometBroadcastExchange + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : +- CometBroadcastExchange : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.item @@ -492,12 +471,11 @@ CometNativeColumnarToRow : : : : +- CometBroadcastHashJoin : : : : :- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- CometSubqueryBroadcast + : : : : : +- CometBroadcastExchange + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : +- CometBroadcastExchange : : : : +- CometBroadcastHashJoin : : : : :- CometFilter @@ -509,12 +487,11 @@ CometNativeColumnarToRow : : : : : +- CometBroadcastHashJoin : : : : : :- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometNativeColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : +- CometSubqueryBroadcast + : : : : : : +- CometBroadcastExchange + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : : +- CometBroadcastExchange : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.item @@ -533,12 +510,11 @@ CometNativeColumnarToRow : : : +- CometBroadcastHashJoin : : : :- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometSubqueryBroadcast + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- CometBroadcastExchange : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.item @@ -567,12 +543,11 @@ CometNativeColumnarToRow : : : :- CometProject : : : : +- CometBroadcastHashJoin : : : : :- CometNativeScan parquet spark_catalog.default.store_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- CometSubqueryBroadcast + : : : : : +- CometBroadcastExchange + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : +- CometBroadcastExchange : : : : +- CometProject : : : : +- CometFilter @@ -580,12 +555,11 @@ CometNativeColumnarToRow : : : :- CometProject : : : : +- CometBroadcastHashJoin : : : : :- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- CometSubqueryBroadcast + : : : : : +- CometBroadcastExchange + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : +- CometBroadcastExchange : : : : +- CometProject : : : : +- CometFilter @@ -593,12 +567,11 @@ CometNativeColumnarToRow : : : +- CometProject : : : +- CometBroadcastHashJoin : : : :- CometNativeScan parquet spark_catalog.default.web_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometSubqueryBroadcast + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter @@ -613,12 +586,11 @@ CometNativeColumnarToRow : : : :- CometBroadcastHashJoin : : : : :- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- CometSubqueryBroadcast + : : : : : +- CometBroadcastExchange + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : +- CometBroadcastExchange : : : : +- CometProject : : : : +- CometBroadcastHashJoin @@ -635,12 +607,11 @@ CometNativeColumnarToRow : : : : : : +- CometBroadcastHashJoin : : : : : : :- CometFilter : : : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : : : : : +- SubqueryBroadcast - : : : : : : : +- BroadcastExchange - : : : : : : : +- CometNativeColumnarToRow - : : : : : : : +- CometProject - : : : : : : : +- CometFilter - : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : : +- CometSubqueryBroadcast + : : : : : : : +- CometBroadcastExchange + : : : : : : : +- CometProject + : : : : : : : +- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : : : +- CometBroadcastExchange : : : : : : +- CometBroadcastHashJoin : : : : : : :- CometFilter @@ -652,12 +623,11 @@ CometNativeColumnarToRow : : : : : : : +- CometBroadcastHashJoin : : : : : : : :- CometFilter : : : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : : : : : +- SubqueryBroadcast - : : : : : : : : +- BroadcastExchange - : : : : : : : : +- CometNativeColumnarToRow - : : : : : : : : +- CometProject - : : : : : : : : +- CometFilter - : : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : : : +- CometSubqueryBroadcast + : : : : : : : : +- CometBroadcastExchange + : : : : : : : : +- CometProject + : : : : : : : : +- CometFilter + : : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : : : : +- CometBroadcastExchange : : : : : : : +- CometFilter : : : : : : : +- CometNativeScan parquet spark_catalog.default.item @@ -676,12 +646,11 @@ CometNativeColumnarToRow : : : : : +- CometBroadcastHashJoin : : : : : :- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometNativeColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : +- CometSubqueryBroadcast + : : : : : : +- CometBroadcastExchange + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : : +- CometBroadcastExchange : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.item @@ -709,12 +678,11 @@ CometNativeColumnarToRow : : : : : +- CometBroadcastHashJoin : : : : : :- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometNativeColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : +- CometSubqueryBroadcast + : : : : : : +- CometBroadcastExchange + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : : +- CometBroadcastExchange : : : : : +- CometBroadcastHashJoin : : : : : :- CometFilter @@ -726,12 +694,11 @@ CometNativeColumnarToRow : : : : : : +- CometBroadcastHashJoin : : : : : : :- CometFilter : : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : : : : +- SubqueryBroadcast - : : : : : : : +- BroadcastExchange - : : : : : : : +- CometNativeColumnarToRow - : : : : : : : +- CometProject - : : : : : : : +- CometFilter - : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : : +- CometSubqueryBroadcast + : : : : : : : +- CometBroadcastExchange + : : : : : : : +- CometProject + : : : : : : : +- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : : : +- CometBroadcastExchange : : : : : : +- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.item @@ -750,12 +717,11 @@ CometNativeColumnarToRow : : : : +- CometBroadcastHashJoin : : : : :- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- CometSubqueryBroadcast + : : : : : +- CometBroadcastExchange + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : +- CometBroadcastExchange : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.item @@ -779,12 +745,11 @@ CometNativeColumnarToRow : : : :- CometBroadcastHashJoin : : : : :- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- CometSubqueryBroadcast + : : : : : +- CometBroadcastExchange + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : +- CometBroadcastExchange : : : : +- CometProject : : : : +- CometBroadcastHashJoin @@ -801,12 +766,11 @@ CometNativeColumnarToRow : : : : : : +- CometBroadcastHashJoin : : : : : : :- CometFilter : : : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : : : : : +- SubqueryBroadcast - : : : : : : : +- BroadcastExchange - : : : : : : : +- CometNativeColumnarToRow - : : : : : : : +- CometProject - : : : : : : : +- CometFilter - : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : : +- CometSubqueryBroadcast + : : : : : : : +- CometBroadcastExchange + : : : : : : : +- CometProject + : : : : : : : +- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : : : +- CometBroadcastExchange : : : : : : +- CometBroadcastHashJoin : : : : : : :- CometFilter @@ -818,12 +782,11 @@ CometNativeColumnarToRow : : : : : : : +- CometBroadcastHashJoin : : : : : : : :- CometFilter : : : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : : : : : +- SubqueryBroadcast - : : : : : : : : +- BroadcastExchange - : : : : : : : : +- CometNativeColumnarToRow - : : : : : : : : +- CometProject - : : : : : : : : +- CometFilter - : : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : : : +- CometSubqueryBroadcast + : : : : : : : : +- CometBroadcastExchange + : : : : : : : : +- CometProject + : : : : : : : : +- CometFilter + : : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : : : : +- CometBroadcastExchange : : : : : : : +- CometFilter : : : : : : : +- CometNativeScan parquet spark_catalog.default.item @@ -842,12 +805,11 @@ CometNativeColumnarToRow : : : : : +- CometBroadcastHashJoin : : : : : :- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometNativeColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : +- CometSubqueryBroadcast + : : : : : : +- CometBroadcastExchange + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : : +- CometBroadcastExchange : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.item @@ -875,12 +837,11 @@ CometNativeColumnarToRow : : : : : +- CometBroadcastHashJoin : : : : : :- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometNativeColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : +- CometSubqueryBroadcast + : : : : : : +- CometBroadcastExchange + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : : +- CometBroadcastExchange : : : : : +- CometBroadcastHashJoin : : : : : :- CometFilter @@ -892,12 +853,11 @@ CometNativeColumnarToRow : : : : : : +- CometBroadcastHashJoin : : : : : : :- CometFilter : : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : : : : +- SubqueryBroadcast - : : : : : : : +- BroadcastExchange - : : : : : : : +- CometNativeColumnarToRow - : : : : : : : +- CometProject - : : : : : : : +- CometFilter - : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : : +- CometSubqueryBroadcast + : : : : : : : +- CometBroadcastExchange + : : : : : : : +- CometProject + : : : : : : : +- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : : : +- CometBroadcastExchange : : : : : : +- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.item @@ -916,12 +876,11 @@ CometNativeColumnarToRow : : : : +- CometBroadcastHashJoin : : : : :- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- CometSubqueryBroadcast + : : : : : +- CometBroadcastExchange + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : +- CometBroadcastExchange : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.item @@ -945,12 +904,11 @@ CometNativeColumnarToRow : : :- CometBroadcastHashJoin : : : :- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometSubqueryBroadcast + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometBroadcastHashJoin @@ -967,12 +925,11 @@ CometNativeColumnarToRow : : : : : +- CometBroadcastHashJoin : : : : : :- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometNativeColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : +- CometSubqueryBroadcast + : : : : : : +- CometBroadcastExchange + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : : +- CometBroadcastExchange : : : : : +- CometBroadcastHashJoin : : : : : :- CometFilter @@ -984,12 +941,11 @@ CometNativeColumnarToRow : : : : : : +- CometBroadcastHashJoin : : : : : : :- CometFilter : : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : : : : +- SubqueryBroadcast - : : : : : : : +- BroadcastExchange - : : : : : : : +- CometNativeColumnarToRow - : : : : : : : +- CometProject - : : : : : : : +- CometFilter - : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : : +- CometSubqueryBroadcast + : : : : : : : +- CometBroadcastExchange + : : : : : : : +- CometProject + : : : : : : : +- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : : : +- CometBroadcastExchange : : : : : : +- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.item @@ -1008,12 +964,11 @@ CometNativeColumnarToRow : : : : +- CometBroadcastHashJoin : : : : :- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- CometSubqueryBroadcast + : : : : : +- CometBroadcastExchange + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : +- CometBroadcastExchange : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.item @@ -1041,12 +996,11 @@ CometNativeColumnarToRow : : : : +- CometBroadcastHashJoin : : : : :- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- CometSubqueryBroadcast + : : : : : +- CometBroadcastExchange + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : +- CometBroadcastExchange : : : : +- CometBroadcastHashJoin : : : : :- CometFilter @@ -1058,12 +1012,11 @@ CometNativeColumnarToRow : : : : : +- CometBroadcastHashJoin : : : : : :- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometNativeColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : +- CometSubqueryBroadcast + : : : : : : +- CometBroadcastExchange + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : : +- CometBroadcastExchange : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.item @@ -1082,12 +1035,11 @@ CometNativeColumnarToRow : : : +- CometBroadcastHashJoin : : : :- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometSubqueryBroadcast + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- CometBroadcastExchange : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.item @@ -1116,12 +1068,11 @@ CometNativeColumnarToRow : : : :- CometProject : : : : +- CometBroadcastHashJoin : : : : :- CometNativeScan parquet spark_catalog.default.store_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- CometSubqueryBroadcast + : : : : : +- CometBroadcastExchange + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : +- CometBroadcastExchange : : : : +- CometProject : : : : +- CometFilter @@ -1129,12 +1080,11 @@ CometNativeColumnarToRow : : : :- CometProject : : : : +- CometBroadcastHashJoin : : : : :- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- CometSubqueryBroadcast + : : : : : +- CometBroadcastExchange + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : +- CometBroadcastExchange : : : : +- CometProject : : : : +- CometFilter @@ -1142,12 +1092,11 @@ CometNativeColumnarToRow : : : +- CometProject : : : +- CometBroadcastHashJoin : : : :- CometNativeScan parquet spark_catalog.default.web_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometSubqueryBroadcast + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter @@ -1162,12 +1111,11 @@ CometNativeColumnarToRow : : : :- CometBroadcastHashJoin : : : : :- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- CometSubqueryBroadcast + : : : : : +- CometBroadcastExchange + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : +- CometBroadcastExchange : : : : +- CometProject : : : : +- CometBroadcastHashJoin @@ -1184,12 +1132,11 @@ CometNativeColumnarToRow : : : : : : +- CometBroadcastHashJoin : : : : : : :- CometFilter : : : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : : : : : +- SubqueryBroadcast - : : : : : : : +- BroadcastExchange - : : : : : : : +- CometNativeColumnarToRow - : : : : : : : +- CometProject - : : : : : : : +- CometFilter - : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : : +- CometSubqueryBroadcast + : : : : : : : +- CometBroadcastExchange + : : : : : : : +- CometProject + : : : : : : : +- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : : : +- CometBroadcastExchange : : : : : : +- CometBroadcastHashJoin : : : : : : :- CometFilter @@ -1201,12 +1148,11 @@ CometNativeColumnarToRow : : : : : : : +- CometBroadcastHashJoin : : : : : : : :- CometFilter : : : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : : : : : +- SubqueryBroadcast - : : : : : : : : +- BroadcastExchange - : : : : : : : : +- CometNativeColumnarToRow - : : : : : : : : +- CometProject - : : : : : : : : +- CometFilter - : : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : : : +- CometSubqueryBroadcast + : : : : : : : : +- CometBroadcastExchange + : : : : : : : : +- CometProject + : : : : : : : : +- CometFilter + : : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : : : : +- CometBroadcastExchange : : : : : : : +- CometFilter : : : : : : : +- CometNativeScan parquet spark_catalog.default.item @@ -1225,12 +1171,11 @@ CometNativeColumnarToRow : : : : : +- CometBroadcastHashJoin : : : : : :- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometNativeColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : +- CometSubqueryBroadcast + : : : : : : +- CometBroadcastExchange + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : : +- CometBroadcastExchange : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.item @@ -1258,12 +1203,11 @@ CometNativeColumnarToRow : : : : : +- CometBroadcastHashJoin : : : : : :- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometNativeColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : +- CometSubqueryBroadcast + : : : : : : +- CometBroadcastExchange + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : : +- CometBroadcastExchange : : : : : +- CometBroadcastHashJoin : : : : : :- CometFilter @@ -1275,12 +1219,11 @@ CometNativeColumnarToRow : : : : : : +- CometBroadcastHashJoin : : : : : : :- CometFilter : : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : : : : +- SubqueryBroadcast - : : : : : : : +- BroadcastExchange - : : : : : : : +- CometNativeColumnarToRow - : : : : : : : +- CometProject - : : : : : : : +- CometFilter - : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : : +- CometSubqueryBroadcast + : : : : : : : +- CometBroadcastExchange + : : : : : : : +- CometProject + : : : : : : : +- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : : : +- CometBroadcastExchange : : : : : : +- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.item @@ -1299,12 +1242,11 @@ CometNativeColumnarToRow : : : : +- CometBroadcastHashJoin : : : : :- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- CometSubqueryBroadcast + : : : : : +- CometBroadcastExchange + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : +- CometBroadcastExchange : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.item @@ -1328,12 +1270,11 @@ CometNativeColumnarToRow : : : :- CometBroadcastHashJoin : : : : :- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- CometSubqueryBroadcast + : : : : : +- CometBroadcastExchange + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : +- CometBroadcastExchange : : : : +- CometProject : : : : +- CometBroadcastHashJoin @@ -1350,12 +1291,11 @@ CometNativeColumnarToRow : : : : : : +- CometBroadcastHashJoin : : : : : : :- CometFilter : : : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : : : : : +- SubqueryBroadcast - : : : : : : : +- BroadcastExchange - : : : : : : : +- CometNativeColumnarToRow - : : : : : : : +- CometProject - : : : : : : : +- CometFilter - : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : : +- CometSubqueryBroadcast + : : : : : : : +- CometBroadcastExchange + : : : : : : : +- CometProject + : : : : : : : +- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : : : +- CometBroadcastExchange : : : : : : +- CometBroadcastHashJoin : : : : : : :- CometFilter @@ -1367,12 +1307,11 @@ CometNativeColumnarToRow : : : : : : : +- CometBroadcastHashJoin : : : : : : : :- CometFilter : : : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : : : : : +- SubqueryBroadcast - : : : : : : : : +- BroadcastExchange - : : : : : : : : +- CometNativeColumnarToRow - : : : : : : : : +- CometProject - : : : : : : : : +- CometFilter - : : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : : : +- CometSubqueryBroadcast + : : : : : : : : +- CometBroadcastExchange + : : : : : : : : +- CometProject + : : : : : : : : +- CometFilter + : : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : : : : +- CometBroadcastExchange : : : : : : : +- CometFilter : : : : : : : +- CometNativeScan parquet spark_catalog.default.item @@ -1391,12 +1330,11 @@ CometNativeColumnarToRow : : : : : +- CometBroadcastHashJoin : : : : : :- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometNativeColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : +- CometSubqueryBroadcast + : : : : : : +- CometBroadcastExchange + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : : +- CometBroadcastExchange : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.item @@ -1424,12 +1362,11 @@ CometNativeColumnarToRow : : : : : +- CometBroadcastHashJoin : : : : : :- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometNativeColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : +- CometSubqueryBroadcast + : : : : : : +- CometBroadcastExchange + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : : +- CometBroadcastExchange : : : : : +- CometBroadcastHashJoin : : : : : :- CometFilter @@ -1441,12 +1378,11 @@ CometNativeColumnarToRow : : : : : : +- CometBroadcastHashJoin : : : : : : :- CometFilter : : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : : : : +- SubqueryBroadcast - : : : : : : : +- BroadcastExchange - : : : : : : : +- CometNativeColumnarToRow - : : : : : : : +- CometProject - : : : : : : : +- CometFilter - : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : : +- CometSubqueryBroadcast + : : : : : : : +- CometBroadcastExchange + : : : : : : : +- CometProject + : : : : : : : +- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : : : +- CometBroadcastExchange : : : : : : +- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.item @@ -1465,12 +1401,11 @@ CometNativeColumnarToRow : : : : +- CometBroadcastHashJoin : : : : :- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- CometSubqueryBroadcast + : : : : : +- CometBroadcastExchange + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : +- CometBroadcastExchange : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.item @@ -1494,12 +1429,11 @@ CometNativeColumnarToRow : : :- CometBroadcastHashJoin : : : :- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometSubqueryBroadcast + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometBroadcastHashJoin @@ -1516,12 +1450,11 @@ CometNativeColumnarToRow : : : : : +- CometBroadcastHashJoin : : : : : :- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometNativeColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : +- CometSubqueryBroadcast + : : : : : : +- CometBroadcastExchange + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : : +- CometBroadcastExchange : : : : : +- CometBroadcastHashJoin : : : : : :- CometFilter @@ -1533,12 +1466,11 @@ CometNativeColumnarToRow : : : : : : +- CometBroadcastHashJoin : : : : : : :- CometFilter : : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : : : : +- SubqueryBroadcast - : : : : : : : +- BroadcastExchange - : : : : : : : +- CometNativeColumnarToRow - : : : : : : : +- CometProject - : : : : : : : +- CometFilter - : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : : +- CometSubqueryBroadcast + : : : : : : : +- CometBroadcastExchange + : : : : : : : +- CometProject + : : : : : : : +- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : : : +- CometBroadcastExchange : : : : : : +- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.item @@ -1557,12 +1489,11 @@ CometNativeColumnarToRow : : : : +- CometBroadcastHashJoin : : : : :- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- CometSubqueryBroadcast + : : : : : +- CometBroadcastExchange + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : +- CometBroadcastExchange : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.item @@ -1590,12 +1521,11 @@ CometNativeColumnarToRow : : : : +- CometBroadcastHashJoin : : : : :- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- CometSubqueryBroadcast + : : : : : +- CometBroadcastExchange + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : +- CometBroadcastExchange : : : : +- CometBroadcastHashJoin : : : : :- CometFilter @@ -1607,12 +1537,11 @@ CometNativeColumnarToRow : : : : : +- CometBroadcastHashJoin : : : : : :- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometNativeColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : +- CometSubqueryBroadcast + : : : : : : +- CometBroadcastExchange + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : : +- CometBroadcastExchange : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.item @@ -1631,12 +1560,11 @@ CometNativeColumnarToRow : : : +- CometBroadcastHashJoin : : : :- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometSubqueryBroadcast + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- CometBroadcastExchange : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.item @@ -1665,12 +1593,11 @@ CometNativeColumnarToRow : : : :- CometProject : : : : +- CometBroadcastHashJoin : : : : :- CometNativeScan parquet spark_catalog.default.store_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- CometSubqueryBroadcast + : : : : : +- CometBroadcastExchange + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : +- CometBroadcastExchange : : : : +- CometProject : : : : +- CometFilter @@ -1678,12 +1605,11 @@ CometNativeColumnarToRow : : : :- CometProject : : : : +- CometBroadcastHashJoin : : : : :- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- CometSubqueryBroadcast + : : : : : +- CometBroadcastExchange + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : +- CometBroadcastExchange : : : : +- CometProject : : : : +- CometFilter @@ -1691,12 +1617,11 @@ CometNativeColumnarToRow : : : +- CometProject : : : +- CometBroadcastHashJoin : : : :- CometNativeScan parquet spark_catalog.default.web_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometSubqueryBroadcast + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter @@ -1711,12 +1636,11 @@ CometNativeColumnarToRow : : : :- CometBroadcastHashJoin : : : : :- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- CometSubqueryBroadcast + : : : : : +- CometBroadcastExchange + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : +- CometBroadcastExchange : : : : +- CometProject : : : : +- CometBroadcastHashJoin @@ -1733,12 +1657,11 @@ CometNativeColumnarToRow : : : : : : +- CometBroadcastHashJoin : : : : : : :- CometFilter : : : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : : : : : +- SubqueryBroadcast - : : : : : : : +- BroadcastExchange - : : : : : : : +- CometNativeColumnarToRow - : : : : : : : +- CometProject - : : : : : : : +- CometFilter - : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : : +- CometSubqueryBroadcast + : : : : : : : +- CometBroadcastExchange + : : : : : : : +- CometProject + : : : : : : : +- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : : : +- CometBroadcastExchange : : : : : : +- CometBroadcastHashJoin : : : : : : :- CometFilter @@ -1750,12 +1673,11 @@ CometNativeColumnarToRow : : : : : : : +- CometBroadcastHashJoin : : : : : : : :- CometFilter : : : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : : : : : +- SubqueryBroadcast - : : : : : : : : +- BroadcastExchange - : : : : : : : : +- CometNativeColumnarToRow - : : : : : : : : +- CometProject - : : : : : : : : +- CometFilter - : : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : : : +- CometSubqueryBroadcast + : : : : : : : : +- CometBroadcastExchange + : : : : : : : : +- CometProject + : : : : : : : : +- CometFilter + : : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : : : : +- CometBroadcastExchange : : : : : : : +- CometFilter : : : : : : : +- CometNativeScan parquet spark_catalog.default.item @@ -1774,12 +1696,11 @@ CometNativeColumnarToRow : : : : : +- CometBroadcastHashJoin : : : : : :- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometNativeColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : +- CometSubqueryBroadcast + : : : : : : +- CometBroadcastExchange + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : : +- CometBroadcastExchange : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.item @@ -1807,12 +1728,11 @@ CometNativeColumnarToRow : : : : : +- CometBroadcastHashJoin : : : : : :- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometNativeColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : +- CometSubqueryBroadcast + : : : : : : +- CometBroadcastExchange + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : : +- CometBroadcastExchange : : : : : +- CometBroadcastHashJoin : : : : : :- CometFilter @@ -1824,12 +1744,11 @@ CometNativeColumnarToRow : : : : : : +- CometBroadcastHashJoin : : : : : : :- CometFilter : : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : : : : +- SubqueryBroadcast - : : : : : : : +- BroadcastExchange - : : : : : : : +- CometNativeColumnarToRow - : : : : : : : +- CometProject - : : : : : : : +- CometFilter - : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : : +- CometSubqueryBroadcast + : : : : : : : +- CometBroadcastExchange + : : : : : : : +- CometProject + : : : : : : : +- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : : : +- CometBroadcastExchange : : : : : : +- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.item @@ -1848,12 +1767,11 @@ CometNativeColumnarToRow : : : : +- CometBroadcastHashJoin : : : : :- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- CometSubqueryBroadcast + : : : : : +- CometBroadcastExchange + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : +- CometBroadcastExchange : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.item @@ -1877,12 +1795,11 @@ CometNativeColumnarToRow : : : :- CometBroadcastHashJoin : : : : :- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- CometSubqueryBroadcast + : : : : : +- CometBroadcastExchange + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : +- CometBroadcastExchange : : : : +- CometProject : : : : +- CometBroadcastHashJoin @@ -1899,12 +1816,11 @@ CometNativeColumnarToRow : : : : : : +- CometBroadcastHashJoin : : : : : : :- CometFilter : : : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : : : : : +- SubqueryBroadcast - : : : : : : : +- BroadcastExchange - : : : : : : : +- CometNativeColumnarToRow - : : : : : : : +- CometProject - : : : : : : : +- CometFilter - : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : : +- CometSubqueryBroadcast + : : : : : : : +- CometBroadcastExchange + : : : : : : : +- CometProject + : : : : : : : +- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : : : +- CometBroadcastExchange : : : : : : +- CometBroadcastHashJoin : : : : : : :- CometFilter @@ -1916,12 +1832,11 @@ CometNativeColumnarToRow : : : : : : : +- CometBroadcastHashJoin : : : : : : : :- CometFilter : : : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : : : : : +- SubqueryBroadcast - : : : : : : : : +- BroadcastExchange - : : : : : : : : +- CometNativeColumnarToRow - : : : : : : : : +- CometProject - : : : : : : : : +- CometFilter - : : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : : : +- CometSubqueryBroadcast + : : : : : : : : +- CometBroadcastExchange + : : : : : : : : +- CometProject + : : : : : : : : +- CometFilter + : : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : : : : +- CometBroadcastExchange : : : : : : : +- CometFilter : : : : : : : +- CometNativeScan parquet spark_catalog.default.item @@ -1940,12 +1855,11 @@ CometNativeColumnarToRow : : : : : +- CometBroadcastHashJoin : : : : : :- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometNativeColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : +- CometSubqueryBroadcast + : : : : : : +- CometBroadcastExchange + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : : +- CometBroadcastExchange : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.item @@ -1973,12 +1887,11 @@ CometNativeColumnarToRow : : : : : +- CometBroadcastHashJoin : : : : : :- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometNativeColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : +- CometSubqueryBroadcast + : : : : : : +- CometBroadcastExchange + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : : +- CometBroadcastExchange : : : : : +- CometBroadcastHashJoin : : : : : :- CometFilter @@ -1990,12 +1903,11 @@ CometNativeColumnarToRow : : : : : : +- CometBroadcastHashJoin : : : : : : :- CometFilter : : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : : : : +- SubqueryBroadcast - : : : : : : : +- BroadcastExchange - : : : : : : : +- CometNativeColumnarToRow - : : : : : : : +- CometProject - : : : : : : : +- CometFilter - : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : : +- CometSubqueryBroadcast + : : : : : : : +- CometBroadcastExchange + : : : : : : : +- CometProject + : : : : : : : +- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : : : +- CometBroadcastExchange : : : : : : +- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.item @@ -2014,12 +1926,11 @@ CometNativeColumnarToRow : : : : +- CometBroadcastHashJoin : : : : :- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- CometSubqueryBroadcast + : : : : : +- CometBroadcastExchange + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : +- CometBroadcastExchange : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.item @@ -2043,12 +1954,11 @@ CometNativeColumnarToRow : : :- CometBroadcastHashJoin : : : :- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometSubqueryBroadcast + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometBroadcastHashJoin @@ -2065,12 +1975,11 @@ CometNativeColumnarToRow : : : : : +- CometBroadcastHashJoin : : : : : :- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometNativeColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : +- CometSubqueryBroadcast + : : : : : : +- CometBroadcastExchange + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : : +- CometBroadcastExchange : : : : : +- CometBroadcastHashJoin : : : : : :- CometFilter @@ -2082,12 +1991,11 @@ CometNativeColumnarToRow : : : : : : +- CometBroadcastHashJoin : : : : : : :- CometFilter : : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : : : : +- SubqueryBroadcast - : : : : : : : +- BroadcastExchange - : : : : : : : +- CometNativeColumnarToRow - : : : : : : : +- CometProject - : : : : : : : +- CometFilter - : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : : +- CometSubqueryBroadcast + : : : : : : : +- CometBroadcastExchange + : : : : : : : +- CometProject + : : : : : : : +- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : : : +- CometBroadcastExchange : : : : : : +- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.item @@ -2106,12 +2014,11 @@ CometNativeColumnarToRow : : : : +- CometBroadcastHashJoin : : : : :- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- CometSubqueryBroadcast + : : : : : +- CometBroadcastExchange + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : +- CometBroadcastExchange : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.item @@ -2139,12 +2046,11 @@ CometNativeColumnarToRow : : : : +- CometBroadcastHashJoin : : : : :- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- CometSubqueryBroadcast + : : : : : +- CometBroadcastExchange + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : +- CometBroadcastExchange : : : : +- CometBroadcastHashJoin : : : : :- CometFilter @@ -2156,12 +2062,11 @@ CometNativeColumnarToRow : : : : : +- CometBroadcastHashJoin : : : : : :- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometNativeColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : +- CometSubqueryBroadcast + : : : : : : +- CometBroadcastExchange + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : : +- CometBroadcastExchange : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.item @@ -2180,12 +2085,11 @@ CometNativeColumnarToRow : : : +- CometBroadcastHashJoin : : : :- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometSubqueryBroadcast + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- CometBroadcastExchange : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.item @@ -2214,12 +2118,11 @@ CometNativeColumnarToRow : : :- CometProject : : : +- CometBroadcastHashJoin : : : :- CometNativeScan parquet spark_catalog.default.store_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometSubqueryBroadcast + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter @@ -2227,12 +2130,11 @@ CometNativeColumnarToRow : : :- CometProject : : : +- CometBroadcastHashJoin : : : :- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometSubqueryBroadcast + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter @@ -2240,12 +2142,11 @@ CometNativeColumnarToRow : : +- CometProject : : +- CometBroadcastHashJoin : : :- CometNativeScan parquet spark_catalog.default.web_sales - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometSubqueryBroadcast + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- CometBroadcastExchange : : +- CometProject : : +- CometFilter @@ -2260,12 +2161,11 @@ CometNativeColumnarToRow : : :- CometBroadcastHashJoin : : : :- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometSubqueryBroadcast + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometBroadcastHashJoin @@ -2282,12 +2182,11 @@ CometNativeColumnarToRow : : : : : +- CometBroadcastHashJoin : : : : : :- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometNativeColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : +- CometSubqueryBroadcast + : : : : : : +- CometBroadcastExchange + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : : +- CometBroadcastExchange : : : : : +- CometBroadcastHashJoin : : : : : :- CometFilter @@ -2299,12 +2198,11 @@ CometNativeColumnarToRow : : : : : : +- CometBroadcastHashJoin : : : : : : :- CometFilter : : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : : : : +- SubqueryBroadcast - : : : : : : : +- BroadcastExchange - : : : : : : : +- CometNativeColumnarToRow - : : : : : : : +- CometProject - : : : : : : : +- CometFilter - : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : : +- CometSubqueryBroadcast + : : : : : : : +- CometBroadcastExchange + : : : : : : : +- CometProject + : : : : : : : +- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : : : +- CometBroadcastExchange : : : : : : +- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.item @@ -2323,12 +2221,11 @@ CometNativeColumnarToRow : : : : +- CometBroadcastHashJoin : : : : :- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- CometSubqueryBroadcast + : : : : : +- CometBroadcastExchange + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : +- CometBroadcastExchange : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.item @@ -2356,12 +2253,11 @@ CometNativeColumnarToRow : : : : +- CometBroadcastHashJoin : : : : :- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- CometSubqueryBroadcast + : : : : : +- CometBroadcastExchange + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : +- CometBroadcastExchange : : : : +- CometBroadcastHashJoin : : : : :- CometFilter @@ -2373,12 +2269,11 @@ CometNativeColumnarToRow : : : : : +- CometBroadcastHashJoin : : : : : :- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometNativeColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : +- CometSubqueryBroadcast + : : : : : : +- CometBroadcastExchange + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : : +- CometBroadcastExchange : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.item @@ -2397,12 +2292,11 @@ CometNativeColumnarToRow : : : +- CometBroadcastHashJoin : : : :- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometSubqueryBroadcast + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- CometBroadcastExchange : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.item @@ -2426,12 +2320,11 @@ CometNativeColumnarToRow : : :- CometBroadcastHashJoin : : : :- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometSubqueryBroadcast + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometBroadcastHashJoin @@ -2448,12 +2341,11 @@ CometNativeColumnarToRow : : : : : +- CometBroadcastHashJoin : : : : : :- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometNativeColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : +- CometSubqueryBroadcast + : : : : : : +- CometBroadcastExchange + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : : +- CometBroadcastExchange : : : : : +- CometBroadcastHashJoin : : : : : :- CometFilter @@ -2465,12 +2357,11 @@ CometNativeColumnarToRow : : : : : : +- CometBroadcastHashJoin : : : : : : :- CometFilter : : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : : : : +- SubqueryBroadcast - : : : : : : : +- BroadcastExchange - : : : : : : : +- CometNativeColumnarToRow - : : : : : : : +- CometProject - : : : : : : : +- CometFilter - : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : : +- CometSubqueryBroadcast + : : : : : : : +- CometBroadcastExchange + : : : : : : : +- CometProject + : : : : : : : +- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : : : +- CometBroadcastExchange : : : : : : +- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.item @@ -2489,12 +2380,11 @@ CometNativeColumnarToRow : : : : +- CometBroadcastHashJoin : : : : :- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- CometSubqueryBroadcast + : : : : : +- CometBroadcastExchange + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : +- CometBroadcastExchange : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.item @@ -2522,12 +2412,11 @@ CometNativeColumnarToRow : : : : +- CometBroadcastHashJoin : : : : :- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- CometSubqueryBroadcast + : : : : : +- CometBroadcastExchange + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : +- CometBroadcastExchange : : : : +- CometBroadcastHashJoin : : : : :- CometFilter @@ -2539,12 +2428,11 @@ CometNativeColumnarToRow : : : : : +- CometBroadcastHashJoin : : : : : :- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometNativeColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : +- CometSubqueryBroadcast + : : : : : : +- CometBroadcastExchange + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : : +- CometBroadcastExchange : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.item @@ -2563,12 +2451,11 @@ CometNativeColumnarToRow : : : +- CometBroadcastHashJoin : : : :- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometSubqueryBroadcast + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- CometBroadcastExchange : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.item @@ -2592,12 +2479,11 @@ CometNativeColumnarToRow : :- CometBroadcastHashJoin : : :- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometSubqueryBroadcast + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- CometBroadcastExchange : : +- CometProject : : +- CometBroadcastHashJoin @@ -2614,12 +2500,11 @@ CometNativeColumnarToRow : : : : +- CometBroadcastHashJoin : : : : :- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- CometSubqueryBroadcast + : : : : : +- CometBroadcastExchange + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : +- CometBroadcastExchange : : : : +- CometBroadcastHashJoin : : : : :- CometFilter @@ -2631,12 +2516,11 @@ CometNativeColumnarToRow : : : : : +- CometBroadcastHashJoin : : : : : :- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometNativeColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : +- CometSubqueryBroadcast + : : : : : : +- CometBroadcastExchange + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : : +- CometBroadcastExchange : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.item @@ -2655,12 +2539,11 @@ CometNativeColumnarToRow : : : +- CometBroadcastHashJoin : : : :- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometSubqueryBroadcast + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- CometBroadcastExchange : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.item @@ -2688,12 +2571,11 @@ CometNativeColumnarToRow : : : +- CometBroadcastHashJoin : : : :- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometSubqueryBroadcast + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- CometBroadcastExchange : : : +- CometBroadcastHashJoin : : : :- CometFilter @@ -2705,12 +2587,11 @@ CometNativeColumnarToRow : : : : +- CometBroadcastHashJoin : : : : :- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- CometSubqueryBroadcast + : : : : : +- CometBroadcastExchange + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : +- CometBroadcastExchange : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.item @@ -2729,12 +2610,11 @@ CometNativeColumnarToRow : : +- CometBroadcastHashJoin : : :- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometSubqueryBroadcast + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- CometBroadcastExchange : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.item @@ -2747,4 +2627,4 @@ CometNativeColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.date_dim -Comet accelerated 2367 out of 2622 eligible operators (90%). Final plan contains 126 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 2487 out of 2622 eligible operators (94%). Final plan contains 6 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q18a.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q18a.native_datafusion/extended.txt index 82bacea2fb..b34ebfdf84 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q18a.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q18a.native_datafusion/extended.txt @@ -18,12 +18,11 @@ CometNativeColumnarToRow : : : : : : +- CometBroadcastHashJoin : : : : : : :- CometFilter : : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : : : : +- SubqueryBroadcast - : : : : : : : +- BroadcastExchange - : : : : : : : +- CometNativeColumnarToRow - : : : : : : : +- CometProject - : : : : : : : +- CometFilter - : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : : +- CometSubqueryBroadcast + : : : : : : : +- CometBroadcastExchange + : : : : : : : +- CometProject + : : : : : : : +- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : : : +- CometBroadcastExchange : : : : : : +- CometProject : : : : : : +- CometFilter @@ -64,12 +63,11 @@ CometNativeColumnarToRow : : : : : : +- CometBroadcastHashJoin : : : : : : :- CometFilter : : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : : : : +- SubqueryBroadcast - : : : : : : : +- BroadcastExchange - : : : : : : : +- CometNativeColumnarToRow - : : : : : : : +- CometProject - : : : : : : : +- CometFilter - : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : : +- CometSubqueryBroadcast + : : : : : : : +- CometBroadcastExchange + : : : : : : : +- CometProject + : : : : : : : +- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : : : +- CometBroadcastExchange : : : : : : +- CometProject : : : : : : +- CometFilter @@ -110,12 +108,11 @@ CometNativeColumnarToRow : : : : : : +- CometBroadcastHashJoin : : : : : : :- CometFilter : : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : : : : +- SubqueryBroadcast - : : : : : : : +- BroadcastExchange - : : : : : : : +- CometNativeColumnarToRow - : : : : : : : +- CometProject - : : : : : : : +- CometFilter - : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : : +- CometSubqueryBroadcast + : : : : : : : +- CometBroadcastExchange + : : : : : : : +- CometProject + : : : : : : : +- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : : : +- CometBroadcastExchange : : : : : : +- CometProject : : : : : : +- CometFilter @@ -156,12 +153,11 @@ CometNativeColumnarToRow : : : : : : +- CometBroadcastHashJoin : : : : : : :- CometFilter : : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : : : : +- SubqueryBroadcast - : : : : : : : +- BroadcastExchange - : : : : : : : +- CometNativeColumnarToRow - : : : : : : : +- CometProject - : : : : : : : +- CometFilter - : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : : +- CometSubqueryBroadcast + : : : : : : : +- CometBroadcastExchange + : : : : : : : +- CometProject + : : : : : : : +- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : : : +- CometBroadcastExchange : : : : : : +- CometProject : : : : : : +- CometFilter @@ -202,12 +198,11 @@ CometNativeColumnarToRow : : : : : +- CometBroadcastHashJoin : : : : : :- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometNativeColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : +- CometSubqueryBroadcast + : : : : : : +- CometBroadcastExchange + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : : +- CometBroadcastExchange : : : : : +- CometProject : : : : : +- CometFilter @@ -231,4 +226,4 @@ CometNativeColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.item -Comet accelerated 216 out of 226 eligible operators (95%). Final plan contains 6 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 221 out of 226 eligible operators (97%). Final plan contains 1 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q20.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q20.native_datafusion/extended.txt index c4f54f4289..24498162ca 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q20.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q20.native_datafusion/extended.txt @@ -13,12 +13,11 @@ TakeOrderedAndProject : +- CometBroadcastHashJoin : :- CometFilter : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometSubqueryBroadcast + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim : +- CometBroadcastExchange : +- CometProject : +- CometFilter @@ -28,4 +27,4 @@ TakeOrderedAndProject +- CometFilter +- CometNativeScan parquet spark_catalog.default.date_dim -Comet accelerated 22 out of 27 eligible operators (81%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 23 out of 27 eligible operators (85%). Final plan contains 1 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q22.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q22.native_datafusion/extended.txt index a8588fd031..5d9c8333d0 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q22.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q22.native_datafusion/extended.txt @@ -13,12 +13,11 @@ TakeOrderedAndProject : : +- CometBroadcastHashJoin : : :- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.inventory - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometSubqueryBroadcast + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- CometBroadcastExchange : : +- CometProject : : +- CometFilter @@ -31,4 +30,4 @@ TakeOrderedAndProject +- CometNativeColumnarToRow +- CometNativeScan parquet spark_catalog.default.warehouse -Comet accelerated 19 out of 28 eligible operators (67%). Final plan contains 4 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 20 out of 28 eligible operators (71%). Final plan contains 3 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q22a.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q22a.native_datafusion/extended.txt index 5bd5daea84..0d8c15e00c 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q22a.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q22a.native_datafusion/extended.txt @@ -14,12 +14,11 @@ CometNativeColumnarToRow : : : +- CometBroadcastHashJoin : : : :- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.inventory - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometSubqueryBroadcast + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter @@ -45,12 +44,11 @@ CometNativeColumnarToRow : : : +- CometBroadcastHashJoin : : : :- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.inventory - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometSubqueryBroadcast + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter @@ -76,12 +74,11 @@ CometNativeColumnarToRow : : : +- CometBroadcastHashJoin : : : :- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.inventory - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometSubqueryBroadcast + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter @@ -107,12 +104,11 @@ CometNativeColumnarToRow : : : +- CometBroadcastHashJoin : : : :- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.inventory - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometSubqueryBroadcast + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter @@ -138,12 +134,11 @@ CometNativeColumnarToRow : : +- CometBroadcastHashJoin : : :- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.inventory - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometSubqueryBroadcast + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- CometBroadcastExchange : : +- CometProject : : +- CometFilter @@ -156,4 +151,4 @@ CometNativeColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.warehouse -Comet accelerated 141 out of 151 eligible operators (93%). Final plan contains 6 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 146 out of 151 eligible operators (96%). Final plan contains 1 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q27a.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q27a.native_datafusion/extended.txt index 5dc06ab86f..9c4497cbd3 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q27a.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q27a.native_datafusion/extended.txt @@ -14,12 +14,11 @@ CometNativeColumnarToRow : : : : +- CometBroadcastHashJoin : : : : :- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- CometSubqueryBroadcast + : : : : : +- CometBroadcastExchange + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : +- CometBroadcastExchange : : : : +- CometProject : : : : +- CometFilter @@ -49,12 +48,11 @@ CometNativeColumnarToRow : : : : +- CometBroadcastHashJoin : : : : :- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- CometSubqueryBroadcast + : : : : : +- CometBroadcastExchange + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : +- CometBroadcastExchange : : : : +- CometProject : : : : +- CometFilter @@ -84,12 +82,11 @@ CometNativeColumnarToRow : : : +- CometBroadcastHashJoin : : : :- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometSubqueryBroadcast + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter @@ -106,4 +103,4 @@ CometNativeColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.item -Comet accelerated 97 out of 103 eligible operators (94%). Final plan contains 4 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 100 out of 103 eligible operators (97%). Final plan contains 1 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q34.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q34.native_datafusion/extended.txt index 1b61be2937..4c0ef9abcd 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q34.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q34.native_datafusion/extended.txt @@ -15,12 +15,11 @@ CometNativeColumnarToRow : : : +- CometBroadcastHashJoin : : : :- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometSubqueryBroadcast + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter @@ -38,4 +37,4 @@ CometNativeColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.customer -Comet accelerated 35 out of 37 eligible operators (94%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 36 out of 37 eligible operators (97%). Final plan contains 1 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q35.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q35.native_datafusion/extended.txt index 2785c3d0b1..2e79f6f464 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q35.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q35.native_datafusion/extended.txt @@ -19,12 +19,11 @@ TakeOrderedAndProject : : : : +- CometProject : : : : +- CometBroadcastHashJoin : : : : :- CometNativeScan parquet spark_catalog.default.store_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- CometSubqueryBroadcast + : : : : : +- CometBroadcastExchange + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : +- CometBroadcastExchange : : : : +- CometProject : : : : +- CometFilter @@ -34,12 +33,11 @@ TakeOrderedAndProject : : : +- CometProject : : : +- CometBroadcastHashJoin : : : :- CometNativeScan parquet spark_catalog.default.web_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometSubqueryBroadcast + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter @@ -49,12 +47,11 @@ TakeOrderedAndProject : : +- CometProject : : +- CometBroadcastHashJoin : : :- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometSubqueryBroadcast + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- CometBroadcastExchange : : +- CometProject : : +- CometFilter @@ -70,4 +67,4 @@ TakeOrderedAndProject +- CometFilter +- CometNativeScan parquet spark_catalog.default.customer_demographics -Comet accelerated 41 out of 62 eligible operators (66%). Final plan contains 9 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 44 out of 62 eligible operators (70%). Final plan contains 6 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q35a.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q35a.native_datafusion/extended.txt index cba125f412..173c450c71 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q35a.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q35a.native_datafusion/extended.txt @@ -16,12 +16,11 @@ CometNativeColumnarToRow : : : +- CometProject : : : +- CometBroadcastHashJoin : : : :- CometNativeScan parquet spark_catalog.default.store_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometSubqueryBroadcast + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter @@ -31,12 +30,11 @@ CometNativeColumnarToRow : : :- CometProject : : : +- CometBroadcastHashJoin : : : :- CometNativeScan parquet spark_catalog.default.web_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometSubqueryBroadcast + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter @@ -44,12 +42,11 @@ CometNativeColumnarToRow : : +- CometProject : : +- CometBroadcastHashJoin : : :- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometSubqueryBroadcast + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- CometBroadcastExchange : : +- CometProject : : +- CometFilter @@ -63,4 +60,4 @@ CometNativeColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.customer_demographics -Comet accelerated 54 out of 60 eligible operators (90%). Final plan contains 4 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 57 out of 60 eligible operators (95%). Final plan contains 1 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q36a.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q36a.native_datafusion/extended.txt index 0a69bca968..be0d09e04e 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q36a.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q36a.native_datafusion/extended.txt @@ -19,12 +19,11 @@ TakeOrderedAndProject : : : +- CometBroadcastHashJoin : : : :- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometSubqueryBroadcast + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter @@ -51,12 +50,11 @@ TakeOrderedAndProject : : : +- CometBroadcastHashJoin : : : :- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometSubqueryBroadcast + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter @@ -83,12 +81,11 @@ TakeOrderedAndProject : : +- CometBroadcastHashJoin : : :- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometSubqueryBroadcast + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- CometBroadcastExchange : : +- CometProject : : +- CometFilter @@ -102,4 +99,4 @@ TakeOrderedAndProject +- CometFilter +- CometNativeScan parquet spark_catalog.default.store -Comet accelerated 90 out of 99 eligible operators (90%). Final plan contains 4 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 93 out of 99 eligible operators (93%). Final plan contains 1 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q47.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q47.native_datafusion/extended.txt index 0886059452..54fa1daf2e 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q47.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q47.native_datafusion/extended.txt @@ -26,11 +26,10 @@ TakeOrderedAndProject : : : : +- CometBroadcastExchange : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometSubqueryBroadcast + : : : : +- CometBroadcastExchange + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- CometBroadcastExchange : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -58,11 +57,10 @@ TakeOrderedAndProject : : : +- CometBroadcastExchange : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometSubqueryBroadcast + : : : +- CometBroadcastExchange + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- CometBroadcastExchange : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -90,11 +88,10 @@ TakeOrderedAndProject : : +- CometBroadcastExchange : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometSubqueryBroadcast + : : +- CometBroadcastExchange + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim : +- CometBroadcastExchange : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -102,4 +99,4 @@ TakeOrderedAndProject +- CometFilter +- CometNativeScan parquet spark_catalog.default.store -Comet accelerated 75 out of 97 eligible operators (77%). Final plan contains 6 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 78 out of 97 eligible operators (80%). Final plan contains 3 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q49.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q49.native_datafusion/extended.txt index 6d7dadcde9..f2ae44fb86 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q49.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q49.native_datafusion/extended.txt @@ -23,12 +23,11 @@ CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometSubqueryBroadcast + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- CometProject : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.web_returns @@ -55,12 +54,11 @@ CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometSubqueryBroadcast + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- CometProject : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.catalog_returns @@ -87,12 +85,11 @@ CometNativeColumnarToRow : : +- CometProject : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometSubqueryBroadcast + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim : +- CometProject : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.store_returns @@ -101,4 +98,4 @@ CometNativeColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.date_dim -Comet accelerated 72 out of 95 eligible operators (75%). Final plan contains 7 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 75 out of 95 eligible operators (78%). Final plan contains 4 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q51a.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q51a.native_datafusion/extended.txt index d9b9186c1e..a3400c5ecf 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q51a.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q51a.native_datafusion/extended.txt @@ -31,12 +31,11 @@ TakeOrderedAndProject : : : +- CometBroadcastHashJoin : : : :- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometSubqueryBroadcast + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter @@ -54,12 +53,11 @@ TakeOrderedAndProject : : +- CometBroadcastHashJoin : : :- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometSubqueryBroadcast + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- CometBroadcastExchange : : +- CometProject : : +- CometFilter @@ -84,12 +82,11 @@ TakeOrderedAndProject : : +- CometBroadcastHashJoin : : :- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometSubqueryBroadcast + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- CometBroadcastExchange : : +- CometProject : : +- CometFilter @@ -107,12 +104,11 @@ TakeOrderedAndProject : +- CometBroadcastHashJoin : :- CometFilter : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometSubqueryBroadcast + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim : +- CometBroadcastExchange : +- CometProject : +- CometFilter @@ -146,12 +142,11 @@ TakeOrderedAndProject : : +- CometBroadcastHashJoin : : :- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometSubqueryBroadcast + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- CometBroadcastExchange : : +- CometProject : : +- CometFilter @@ -169,12 +164,11 @@ TakeOrderedAndProject : +- CometBroadcastHashJoin : :- CometFilter : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometSubqueryBroadcast + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim : +- CometBroadcastExchange : +- CometProject : +- CometFilter @@ -199,12 +193,11 @@ TakeOrderedAndProject : +- CometBroadcastHashJoin : :- CometFilter : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometSubqueryBroadcast + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim : +- CometBroadcastExchange : +- CometProject : +- CometFilter @@ -222,15 +215,14 @@ TakeOrderedAndProject +- CometBroadcastHashJoin :- CometFilter : +- CometNativeScan parquet spark_catalog.default.store_sales - : +- SubqueryBroadcast - : +- BroadcastExchange - : +- CometNativeColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometSubqueryBroadcast + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.date_dim +- CometBroadcastExchange +- CometProject +- CometFilter +- CometNativeScan parquet spark_catalog.default.date_dim -Comet accelerated 150 out of 212 eligible operators (70%). Final plan contains 22 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 158 out of 212 eligible operators (74%). Final plan contains 14 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q57.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q57.native_datafusion/extended.txt index 024bfeeeea..aa4ccf3246 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q57.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q57.native_datafusion/extended.txt @@ -26,11 +26,10 @@ TakeOrderedAndProject : : : : +- CometBroadcastExchange : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometSubqueryBroadcast + : : : : +- CometBroadcastExchange + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- CometBroadcastExchange : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -58,11 +57,10 @@ TakeOrderedAndProject : : : +- CometBroadcastExchange : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometSubqueryBroadcast + : : : +- CometBroadcastExchange + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- CometBroadcastExchange : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -90,11 +88,10 @@ TakeOrderedAndProject : : +- CometBroadcastExchange : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometSubqueryBroadcast + : : +- CometBroadcastExchange + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim : +- CometBroadcastExchange : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -102,4 +99,4 @@ TakeOrderedAndProject +- CometFilter +- CometNativeScan parquet spark_catalog.default.call_center -Comet accelerated 75 out of 97 eligible operators (77%). Final plan contains 6 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 78 out of 97 eligible operators (80%). Final plan contains 3 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q5a.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q5a.native_datafusion/extended.txt index 9cc06fb361..b472530c10 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q5a.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q5a.native_datafusion/extended.txt @@ -19,21 +19,19 @@ CometNativeColumnarToRow : : : : :- CometProject : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- CometSubqueryBroadcast + : : : : : +- CometBroadcastExchange + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : +- CometProject : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.store_returns - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometSubqueryBroadcast + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter @@ -53,21 +51,19 @@ CometNativeColumnarToRow : : : : :- CometProject : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- CometSubqueryBroadcast + : : : : : +- CometBroadcastExchange + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : +- CometProject : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.catalog_returns - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometSubqueryBroadcast + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter @@ -87,22 +83,20 @@ CometNativeColumnarToRow : : : :- CometProject : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometSubqueryBroadcast + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- CometProject : : : +- CometBroadcastHashJoin : : : :- CometBroadcastExchange : : : : +- CometNativeScan parquet spark_catalog.default.web_returns - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometSubqueryBroadcast + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.web_sales @@ -132,21 +126,19 @@ CometNativeColumnarToRow : : : : :- CometProject : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- CometSubqueryBroadcast + : : : : : +- CometBroadcastExchange + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : +- CometProject : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.store_returns - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometSubqueryBroadcast + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter @@ -166,21 +158,19 @@ CometNativeColumnarToRow : : : : :- CometProject : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- CometSubqueryBroadcast + : : : : : +- CometBroadcastExchange + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : +- CometProject : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.catalog_returns - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometSubqueryBroadcast + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter @@ -200,22 +190,20 @@ CometNativeColumnarToRow : : : :- CometProject : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometSubqueryBroadcast + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- CometProject : : : +- CometBroadcastHashJoin : : : :- CometBroadcastExchange : : : : +- CometNativeScan parquet spark_catalog.default.web_returns - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometSubqueryBroadcast + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.web_sales @@ -245,21 +233,19 @@ CometNativeColumnarToRow : : : :- CometProject : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometSubqueryBroadcast + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.store_returns - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometSubqueryBroadcast + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- CometBroadcastExchange : : +- CometProject : : +- CometFilter @@ -279,21 +265,19 @@ CometNativeColumnarToRow : : : :- CometProject : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometSubqueryBroadcast + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.catalog_returns - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometSubqueryBroadcast + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- CometBroadcastExchange : : +- CometProject : : +- CometFilter @@ -313,22 +297,20 @@ CometNativeColumnarToRow : : :- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometSubqueryBroadcast + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- CometProject : : +- CometBroadcastHashJoin : : :- CometBroadcastExchange : : : +- CometNativeScan parquet spark_catalog.default.web_returns - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometSubqueryBroadcast + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- CometProject : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.web_sales @@ -341,4 +323,4 @@ CometNativeColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.web_site -Comet accelerated 287 out of 323 eligible operators (88%). Final plan contains 19 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 305 out of 323 eligible operators (94%). Final plan contains 1 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q6.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q6.native_datafusion/extended.txt index d5de400618..e8759a6e10 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q6.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q6.native_datafusion/extended.txt @@ -21,28 +21,27 @@ CometNativeColumnarToRow : : +- CometBroadcastExchange : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometProject - : : +- CometFilter - : : : +- Subquery - : : : +- CometNativeColumnarToRow - : : : +- CometHashAggregate - : : : +- CometExchange - : : : +- CometHashAggregate - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- Subquery - : : +- CometNativeColumnarToRow - : : +- CometHashAggregate - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometSubqueryBroadcast + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : : +- Subquery + : : : +- CometNativeColumnarToRow + : : : +- CometHashAggregate + : : : +- CometExchange + : : : +- CometHashAggregate + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- Subquery + : : +- CometNativeColumnarToRow + : : +- CometHashAggregate + : : +- CometExchange + : : +- CometHashAggregate + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim : +- CometBroadcastExchange : +- CometProject : +- CometFilter @@ -70,4 +69,4 @@ CometNativeColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.item -Comet accelerated 60 out of 66 eligible operators (90%). Final plan contains 5 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 61 out of 66 eligible operators (92%). Final plan contains 4 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q64.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q64.native_datafusion/extended.txt index f1fe6fb3e3..14ffe4eab9 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q64.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q64.native_datafusion/extended.txt @@ -46,11 +46,10 @@ CometNativeColumnarToRow : : : : : : : : : : : : : : : : : :- CometBroadcastExchange : : : : : : : : : : : : : : : : : : +- CometFilter : : : : : : : : : : : : : : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : : : : : : : : : : : : : : : : +- SubqueryBroadcast - : : : : : : : : : : : : : : : : : : +- BroadcastExchange - : : : : : : : : : : : : : : : : : : +- CometNativeColumnarToRow - : : : : : : : : : : : : : : : : : : +- CometFilter - : : : : : : : : : : : : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : : : : : : : : : : : : : +- CometSubqueryBroadcast + : : : : : : : : : : : : : : : : : : +- CometBroadcastExchange + : : : : : : : : : : : : : : : : : : +- CometFilter + : : : : : : : : : : : : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : : : : : : : : : : : : : : +- CometProject : : : : : : : : : : : : : : : : : +- CometFilter : : : : : : : : : : : : : : : : : +- CometNativeScan parquet spark_catalog.default.store_returns @@ -166,11 +165,10 @@ CometNativeColumnarToRow : : : : : : : : : : : : : : : : :- CometBroadcastExchange : : : : : : : : : : : : : : : : : +- CometFilter : : : : : : : : : : : : : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : : : : : : : : : : : : : : : +- SubqueryBroadcast - : : : : : : : : : : : : : : : : : +- BroadcastExchange - : : : : : : : : : : : : : : : : : +- CometNativeColumnarToRow - : : : : : : : : : : : : : : : : : +- CometFilter - : : : : : : : : : : : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : : : : : : : : : : : : +- CometSubqueryBroadcast + : : : : : : : : : : : : : : : : : +- CometBroadcastExchange + : : : : : : : : : : : : : : : : : +- CometFilter + : : : : : : : : : : : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : : : : : : : : : : : : : +- CometProject : : : : : : : : : : : : : : : : +- CometFilter : : : : : : : : : : : : : : : : +- CometNativeScan parquet spark_catalog.default.store_returns @@ -244,4 +242,4 @@ CometNativeColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.item -Comet accelerated 238 out of 242 eligible operators (98%). Final plan contains 3 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 240 out of 242 eligible operators (99%). Final plan contains 1 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q67a.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q67a.native_datafusion/extended.txt index abf8dad401..8abd1fc15f 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q67a.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q67a.native_datafusion/extended.txt @@ -20,12 +20,11 @@ TakeOrderedAndProject : : : +- CometBroadcastHashJoin : : : :- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometSubqueryBroadcast + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter @@ -52,12 +51,11 @@ TakeOrderedAndProject : : : +- CometBroadcastHashJoin : : : :- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometSubqueryBroadcast + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter @@ -84,12 +82,11 @@ TakeOrderedAndProject : : : +- CometBroadcastHashJoin : : : :- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometSubqueryBroadcast + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter @@ -116,12 +113,11 @@ TakeOrderedAndProject : : : +- CometBroadcastHashJoin : : : :- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometSubqueryBroadcast + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter @@ -148,12 +144,11 @@ TakeOrderedAndProject : : : +- CometBroadcastHashJoin : : : :- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometSubqueryBroadcast + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter @@ -180,12 +175,11 @@ TakeOrderedAndProject : : : +- CometBroadcastHashJoin : : : :- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometSubqueryBroadcast + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter @@ -212,12 +206,11 @@ TakeOrderedAndProject : : : +- CometBroadcastHashJoin : : : :- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometSubqueryBroadcast + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter @@ -244,12 +237,11 @@ TakeOrderedAndProject : : : +- CometBroadcastHashJoin : : : :- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometSubqueryBroadcast + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter @@ -276,12 +268,11 @@ TakeOrderedAndProject : : +- CometBroadcastHashJoin : : :- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometSubqueryBroadcast + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- CometBroadcastExchange : : +- CometProject : : +- CometFilter @@ -295,4 +286,4 @@ TakeOrderedAndProject +- CometFilter +- CometNativeScan parquet spark_catalog.default.item -Comet accelerated 262 out of 285 eligible operators (91%). Final plan contains 11 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 271 out of 285 eligible operators (95%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q70a.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q70a.native_datafusion/extended.txt index 4ba9e6eb00..97dd294f61 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q70a.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q70a.native_datafusion/extended.txt @@ -19,12 +19,11 @@ TakeOrderedAndProject : : +- CometBroadcastHashJoin : : :- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometSubqueryBroadcast + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- CometBroadcastExchange : : +- CometProject : : +- CometFilter @@ -51,12 +50,11 @@ TakeOrderedAndProject : : +- CometBroadcastHashJoin : : :- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometSubqueryBroadcast + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- CometBroadcastExchange : : +- CometProject : : +- CometFilter @@ -80,12 +78,11 @@ TakeOrderedAndProject : : +- CometBroadcastHashJoin : : :- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometSubqueryBroadcast + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- CometBroadcastExchange : : +- CometProject : : +- CometFilter @@ -112,12 +109,11 @@ TakeOrderedAndProject : : +- CometBroadcastHashJoin : : :- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometSubqueryBroadcast + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- CometBroadcastExchange : : +- CometProject : : +- CometFilter @@ -141,12 +137,11 @@ TakeOrderedAndProject : +- CometBroadcastHashJoin : :- CometFilter : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometSubqueryBroadcast + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim : +- CometBroadcastExchange : +- CometProject : +- CometFilter @@ -173,12 +168,11 @@ TakeOrderedAndProject : +- CometBroadcastHashJoin : :- CometFilter : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometSubqueryBroadcast + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim : +- CometBroadcastExchange : +- CometProject : +- CometFilter @@ -188,4 +182,4 @@ TakeOrderedAndProject +- CometFilter +- CometNativeScan parquet spark_catalog.default.date_dim -Comet accelerated 111 out of 168 eligible operators (66%). Final plan contains 21 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 117 out of 168 eligible operators (69%). Final plan contains 15 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q72.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q72.native_datafusion/extended.txt index 623a68ddab..d443b31d1e 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q72.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q72.native_datafusion/extended.txt @@ -27,12 +27,11 @@ CometNativeColumnarToRow : : : : : : : : : +- CometBroadcastHashJoin : : : : : : : : : :- CometFilter : : : : : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : : : : : : : +- SubqueryBroadcast - : : : : : : : : : : +- BroadcastExchange - : : : : : : : : : : +- CometNativeColumnarToRow - : : : : : : : : : : +- CometProject - : : : : : : : : : : +- CometFilter - : : : : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : : : : : +- CometSubqueryBroadcast + : : : : : : : : : : +- CometBroadcastExchange + : : : : : : : : : : +- CometProject + : : : : : : : : : : +- CometFilter + : : : : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : : : : : : +- CometBroadcastExchange : : : : : : : : : +- CometFilter : : : : : : : : : +- CometNativeScan parquet spark_catalog.default.inventory @@ -69,4 +68,4 @@ CometNativeColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.catalog_returns -Comet accelerated 66 out of 68 eligible operators (97%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 67 out of 68 eligible operators (98%). Final plan contains 1 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q74.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q74.native_datafusion/extended.txt index 51a3f5ca94..7768138645 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q74.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q74.native_datafusion/extended.txt @@ -19,11 +19,10 @@ CometNativeColumnarToRow : : : : +- CometBroadcastExchange : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometSubqueryBroadcast + : : : : +- CometBroadcastExchange + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- CometBroadcastExchange : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -41,11 +40,10 @@ CometNativeColumnarToRow : : : +- CometBroadcastExchange : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometSubqueryBroadcast + : : : +- CometBroadcastExchange + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- CometBroadcastExchange : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -64,11 +62,10 @@ CometNativeColumnarToRow : : +- CometBroadcastExchange : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometSubqueryBroadcast + : : +- CometBroadcastExchange + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim : +- CometBroadcastExchange : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -86,13 +83,12 @@ CometNativeColumnarToRow : +- CometBroadcastExchange : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.web_sales - : +- SubqueryBroadcast - : +- BroadcastExchange - : +- CometNativeColumnarToRow - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometSubqueryBroadcast + : +- CometBroadcastExchange + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.date_dim +- CometBroadcastExchange +- CometFilter +- CometNativeScan parquet spark_catalog.default.date_dim -Comet accelerated 83 out of 91 eligible operators (91%). Final plan contains 5 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 87 out of 91 eligible operators (95%). Final plan contains 1 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q75.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q75.native_datafusion/extended.txt index a943308f1e..ee7c0af216 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q75.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q75.native_datafusion/extended.txt @@ -22,11 +22,10 @@ CometNativeColumnarToRow : : : : +- CometBroadcastHashJoin : : : : :- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- CometSubqueryBroadcast + : : : : : +- CometBroadcastExchange + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : +- CometBroadcastExchange : : : : +- CometProject : : : : +- CometFilter @@ -49,11 +48,10 @@ CometNativeColumnarToRow : : : : +- CometBroadcastHashJoin : : : : :- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- CometSubqueryBroadcast + : : : : : +- CometBroadcastExchange + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : +- CometBroadcastExchange : : : : +- CometProject : : : : +- CometFilter @@ -76,11 +74,10 @@ CometNativeColumnarToRow : : : +- CometBroadcastHashJoin : : : :- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometSubqueryBroadcast + : : : : +- CometBroadcastExchange + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter @@ -113,11 +110,10 @@ CometNativeColumnarToRow : : : +- CometBroadcastHashJoin : : : :- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometSubqueryBroadcast + : : : : +- CometBroadcastExchange + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter @@ -140,11 +136,10 @@ CometNativeColumnarToRow : : : +- CometBroadcastHashJoin : : : :- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometSubqueryBroadcast + : : : : +- CometBroadcastExchange + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter @@ -167,11 +162,10 @@ CometNativeColumnarToRow : : +- CometBroadcastHashJoin : : :- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometSubqueryBroadcast + : : : +- CometBroadcastExchange + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- CometBroadcastExchange : : +- CometProject : : +- CometFilter @@ -185,4 +179,4 @@ CometNativeColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.web_returns -Comet accelerated 167 out of 179 eligible operators (93%). Final plan contains 7 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 173 out of 179 eligible operators (96%). Final plan contains 1 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q77a.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q77a.native_datafusion/extended.txt index 24407fc0a2..6621cfd8f8 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q77a.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q77a.native_datafusion/extended.txt @@ -21,12 +21,11 @@ CometNativeColumnarToRow : : : : +- CometBroadcastHashJoin : : : : :- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- CometSubqueryBroadcast + : : : : : +- CometBroadcastExchange + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : +- CometBroadcastExchange : : : : +- CometProject : : : : +- CometFilter @@ -44,12 +43,11 @@ CometNativeColumnarToRow : : : +- CometBroadcastHashJoin : : : :- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.store_returns - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometSubqueryBroadcast + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter @@ -67,12 +65,11 @@ CometNativeColumnarToRow : : : +- CometProject : : : +- CometBroadcastHashJoin : : : :- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometSubqueryBroadcast + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter @@ -84,12 +81,11 @@ CometNativeColumnarToRow : : +- CometProject : : +- CometBroadcastHashJoin : : :- CometNativeScan parquet spark_catalog.default.catalog_returns - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometSubqueryBroadcast + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- CometBroadcastExchange : : +- CometProject : : +- CometFilter @@ -106,12 +102,11 @@ CometNativeColumnarToRow : : : +- CometBroadcastHashJoin : : : :- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometSubqueryBroadcast + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter @@ -129,12 +124,11 @@ CometNativeColumnarToRow : : +- CometBroadcastHashJoin : : :- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.web_returns - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometSubqueryBroadcast + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- CometBroadcastExchange : : +- CometProject : : +- CometFilter @@ -163,12 +157,11 @@ CometNativeColumnarToRow : : : : +- CometBroadcastHashJoin : : : : :- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- CometSubqueryBroadcast + : : : : : +- CometBroadcastExchange + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : +- CometBroadcastExchange : : : : +- CometProject : : : : +- CometFilter @@ -186,12 +179,11 @@ CometNativeColumnarToRow : : : +- CometBroadcastHashJoin : : : :- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.store_returns - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometSubqueryBroadcast + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter @@ -209,12 +201,11 @@ CometNativeColumnarToRow : : : +- CometProject : : : +- CometBroadcastHashJoin : : : :- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometSubqueryBroadcast + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter @@ -226,12 +217,11 @@ CometNativeColumnarToRow : : +- CometProject : : +- CometBroadcastHashJoin : : :- CometNativeScan parquet spark_catalog.default.catalog_returns - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometSubqueryBroadcast + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- CometBroadcastExchange : : +- CometProject : : +- CometFilter @@ -248,12 +238,11 @@ CometNativeColumnarToRow : : : +- CometBroadcastHashJoin : : : :- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometSubqueryBroadcast + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter @@ -271,12 +260,11 @@ CometNativeColumnarToRow : : +- CometBroadcastHashJoin : : :- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.web_returns - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometSubqueryBroadcast + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- CometBroadcastExchange : : +- CometProject : : +- CometFilter @@ -305,12 +293,11 @@ CometNativeColumnarToRow : : : +- CometBroadcastHashJoin : : : :- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometSubqueryBroadcast + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter @@ -328,12 +315,11 @@ CometNativeColumnarToRow : : +- CometBroadcastHashJoin : : :- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.store_returns - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometSubqueryBroadcast + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- CometBroadcastExchange : : +- CometProject : : +- CometFilter @@ -351,12 +337,11 @@ CometNativeColumnarToRow : : +- CometProject : : +- CometBroadcastHashJoin : : :- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometSubqueryBroadcast + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- CometBroadcastExchange : : +- CometProject : : +- CometFilter @@ -368,12 +353,11 @@ CometNativeColumnarToRow : +- CometProject : +- CometBroadcastHashJoin : :- CometNativeScan parquet spark_catalog.default.catalog_returns - : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometSubqueryBroadcast + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim : +- CometBroadcastExchange : +- CometProject : +- CometFilter @@ -390,12 +374,11 @@ CometNativeColumnarToRow : : +- CometBroadcastHashJoin : : :- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometSubqueryBroadcast + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- CometBroadcastExchange : : +- CometProject : : +- CometFilter @@ -413,12 +396,11 @@ CometNativeColumnarToRow : +- CometBroadcastHashJoin : :- CometFilter : : +- CometNativeScan parquet spark_catalog.default.web_returns - : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometSubqueryBroadcast + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim : +- CometBroadcastExchange : +- CometProject : +- CometFilter @@ -427,4 +409,4 @@ CometNativeColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.web_page -Comet accelerated 332 out of 392 eligible operators (84%). Final plan contains 36 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 350 out of 392 eligible operators (89%). Final plan contains 18 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q78.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q78.native_datafusion/extended.txt index ebfdcc3c62..8426b9653c 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q78.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q78.native_datafusion/extended.txt @@ -17,11 +17,10 @@ TakeOrderedAndProject : : : : +- CometExchange : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometSubqueryBroadcast + : : : : +- CometBroadcastExchange + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- CometSort : : : +- CometExchange : : : +- CometProject @@ -44,11 +43,10 @@ TakeOrderedAndProject : : : +- CometExchange : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometSubqueryBroadcast + : : : +- CometBroadcastExchange + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- CometSort : : +- CometExchange : : +- CometProject @@ -71,11 +69,10 @@ TakeOrderedAndProject : : +- CometExchange : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometSubqueryBroadcast + : : +- CometBroadcastExchange + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim : +- CometSort : +- CometExchange : +- CometProject @@ -85,4 +82,4 @@ TakeOrderedAndProject +- CometFilter +- CometNativeScan parquet spark_catalog.default.date_dim -Comet accelerated 74 out of 82 eligible operators (90%). Final plan contains 4 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 77 out of 82 eligible operators (93%). Final plan contains 1 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q80a.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q80a.native_datafusion/extended.txt index ec566b42de..dced05333a 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q80a.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q80a.native_datafusion/extended.txt @@ -25,12 +25,11 @@ CometNativeColumnarToRow : : : : : : : +- CometExchange : : : : : : : +- CometFilter : : : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : : : : : +- SubqueryBroadcast - : : : : : : : +- BroadcastExchange - : : : : : : : +- CometNativeColumnarToRow - : : : : : : : +- CometProject - : : : : : : : +- CometFilter - : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : : +- CometSubqueryBroadcast + : : : : : : : +- CometBroadcastExchange + : : : : : : : +- CometProject + : : : : : : : +- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : : : +- CometSort : : : : : : +- CometExchange : : : : : : +- CometProject @@ -69,12 +68,11 @@ CometNativeColumnarToRow : : : : : : : +- CometExchange : : : : : : : +- CometFilter : : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : : : : +- SubqueryBroadcast - : : : : : : : +- BroadcastExchange - : : : : : : : +- CometNativeColumnarToRow - : : : : : : : +- CometProject - : : : : : : : +- CometFilter - : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : : +- CometSubqueryBroadcast + : : : : : : : +- CometBroadcastExchange + : : : : : : : +- CometProject + : : : : : : : +- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : : : +- CometSort : : : : : : +- CometExchange : : : : : : +- CometProject @@ -113,12 +111,11 @@ CometNativeColumnarToRow : : : : : : +- CometExchange : : : : : : +- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometNativeColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : +- CometSubqueryBroadcast + : : : : : : +- CometBroadcastExchange + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : : +- CometSort : : : : : +- CometExchange : : : : : +- CometProject @@ -164,12 +161,11 @@ CometNativeColumnarToRow : : : : : : : +- CometExchange : : : : : : : +- CometFilter : : : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : : : : : +- SubqueryBroadcast - : : : : : : : +- BroadcastExchange - : : : : : : : +- CometNativeColumnarToRow - : : : : : : : +- CometProject - : : : : : : : +- CometFilter - : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : : +- CometSubqueryBroadcast + : : : : : : : +- CometBroadcastExchange + : : : : : : : +- CometProject + : : : : : : : +- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : : : +- CometSort : : : : : : +- CometExchange : : : : : : +- CometProject @@ -208,12 +204,11 @@ CometNativeColumnarToRow : : : : : : : +- CometExchange : : : : : : : +- CometFilter : : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : : : : +- SubqueryBroadcast - : : : : : : : +- BroadcastExchange - : : : : : : : +- CometNativeColumnarToRow - : : : : : : : +- CometProject - : : : : : : : +- CometFilter - : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : : +- CometSubqueryBroadcast + : : : : : : : +- CometBroadcastExchange + : : : : : : : +- CometProject + : : : : : : : +- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : : : +- CometSort : : : : : : +- CometExchange : : : : : : +- CometProject @@ -252,12 +247,11 @@ CometNativeColumnarToRow : : : : : : +- CometExchange : : : : : : +- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometNativeColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : +- CometSubqueryBroadcast + : : : : : : +- CometBroadcastExchange + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : : +- CometSort : : : : : +- CometExchange : : : : : +- CometProject @@ -303,12 +297,11 @@ CometNativeColumnarToRow : : : : : : +- CometExchange : : : : : : +- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometNativeColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : +- CometSubqueryBroadcast + : : : : : : +- CometBroadcastExchange + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : : +- CometSort : : : : : +- CometExchange : : : : : +- CometProject @@ -347,12 +340,11 @@ CometNativeColumnarToRow : : : : : : +- CometExchange : : : : : : +- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometNativeColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : +- CometSubqueryBroadcast + : : : : : : +- CometBroadcastExchange + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : : +- CometSort : : : : : +- CometExchange : : : : : +- CometProject @@ -391,12 +383,11 @@ CometNativeColumnarToRow : : : : : +- CometExchange : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- CometSubqueryBroadcast + : : : : : +- CometBroadcastExchange + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : +- CometSort : : : : +- CometExchange : : : : +- CometProject @@ -419,4 +410,4 @@ CometNativeColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.promotion -Comet accelerated 392 out of 410 eligible operators (95%). Final plan contains 10 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 401 out of 410 eligible operators (97%). Final plan contains 1 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q86a.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q86a.native_datafusion/extended.txt index c1df720935..4c66209a5e 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q86a.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q86a.native_datafusion/extended.txt @@ -17,12 +17,11 @@ TakeOrderedAndProject : : +- CometBroadcastHashJoin : : :- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometSubqueryBroadcast + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- CometBroadcastExchange : : +- CometProject : : +- CometFilter @@ -43,12 +42,11 @@ TakeOrderedAndProject : : +- CometBroadcastHashJoin : : :- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometSubqueryBroadcast + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- CometBroadcastExchange : : +- CometProject : : +- CometFilter @@ -69,12 +67,11 @@ TakeOrderedAndProject : +- CometBroadcastHashJoin : :- CometFilter : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometSubqueryBroadcast + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim : +- CometBroadcastExchange : +- CometProject : +- CometFilter @@ -84,4 +81,4 @@ TakeOrderedAndProject +- CometFilter +- CometNativeScan parquet spark_catalog.default.item -Comet accelerated 72 out of 81 eligible operators (88%). Final plan contains 4 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 75 out of 81 eligible operators (92%). Final plan contains 1 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q98.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q98.native_datafusion/extended.txt index 6c2b043008..ccf9f25336 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q98.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q98.native_datafusion/extended.txt @@ -15,12 +15,11 @@ CometNativeColumnarToRow : +- CometBroadcastHashJoin : :- CometFilter : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometSubqueryBroadcast + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim : +- CometBroadcastExchange : +- CometProject : +- CometFilter @@ -30,4 +29,4 @@ CometNativeColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.date_dim -Comet accelerated 24 out of 28 eligible operators (85%). Final plan contains 3 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 25 out of 28 eligible operators (89%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q10a.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q10a.native_datafusion/extended.txt index cba125f412..173c450c71 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q10a.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q10a.native_datafusion/extended.txt @@ -16,12 +16,11 @@ CometNativeColumnarToRow : : : +- CometProject : : : +- CometBroadcastHashJoin : : : :- CometNativeScan parquet spark_catalog.default.store_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometSubqueryBroadcast + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter @@ -31,12 +30,11 @@ CometNativeColumnarToRow : : :- CometProject : : : +- CometBroadcastHashJoin : : : :- CometNativeScan parquet spark_catalog.default.web_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometSubqueryBroadcast + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter @@ -44,12 +42,11 @@ CometNativeColumnarToRow : : +- CometProject : : +- CometBroadcastHashJoin : : :- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometSubqueryBroadcast + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- CometBroadcastExchange : : +- CometProject : : +- CometFilter @@ -63,4 +60,4 @@ CometNativeColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.customer_demographics -Comet accelerated 54 out of 60 eligible operators (90%). Final plan contains 4 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 57 out of 60 eligible operators (95%). Final plan contains 1 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q11.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q11.native_datafusion/extended.txt index 51a3f5ca94..7768138645 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q11.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q11.native_datafusion/extended.txt @@ -19,11 +19,10 @@ CometNativeColumnarToRow : : : : +- CometBroadcastExchange : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometSubqueryBroadcast + : : : : +- CometBroadcastExchange + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- CometBroadcastExchange : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -41,11 +40,10 @@ CometNativeColumnarToRow : : : +- CometBroadcastExchange : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometSubqueryBroadcast + : : : +- CometBroadcastExchange + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- CometBroadcastExchange : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -64,11 +62,10 @@ CometNativeColumnarToRow : : +- CometBroadcastExchange : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometSubqueryBroadcast + : : +- CometBroadcastExchange + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim : +- CometBroadcastExchange : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -86,13 +83,12 @@ CometNativeColumnarToRow : +- CometBroadcastExchange : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.web_sales - : +- SubqueryBroadcast - : +- BroadcastExchange - : +- CometNativeColumnarToRow - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometSubqueryBroadcast + : +- CometBroadcastExchange + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.date_dim +- CometBroadcastExchange +- CometFilter +- CometNativeScan parquet spark_catalog.default.date_dim -Comet accelerated 83 out of 91 eligible operators (91%). Final plan contains 5 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 87 out of 91 eligible operators (95%). Final plan contains 1 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q12.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q12.native_datafusion/extended.txt index a8c2531a38..a6cc526735 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q12.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q12.native_datafusion/extended.txt @@ -13,12 +13,11 @@ TakeOrderedAndProject : +- CometBroadcastHashJoin : :- CometFilter : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometSubqueryBroadcast + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim : +- CometBroadcastExchange : +- CometProject : +- CometFilter @@ -28,4 +27,4 @@ TakeOrderedAndProject +- CometFilter +- CometNativeScan parquet spark_catalog.default.date_dim -Comet accelerated 22 out of 27 eligible operators (81%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 23 out of 27 eligible operators (85%). Final plan contains 1 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14.native_datafusion/extended.txt index 50139381f2..2d20c40e4b 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14.native_datafusion/extended.txt @@ -11,12 +11,11 @@ CometNativeColumnarToRow : : :- CometProject : : : +- CometBroadcastHashJoin : : : :- CometNativeScan parquet spark_catalog.default.store_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometSubqueryBroadcast + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter @@ -24,12 +23,11 @@ CometNativeColumnarToRow : : :- CometProject : : : +- CometBroadcastHashJoin : : : :- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometSubqueryBroadcast + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter @@ -37,12 +35,11 @@ CometNativeColumnarToRow : : +- CometProject : : +- CometBroadcastHashJoin : : :- CometNativeScan parquet spark_catalog.default.web_sales - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometSubqueryBroadcast + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- CometBroadcastExchange : : +- CometProject : : +- CometFilter @@ -57,17 +54,16 @@ CometNativeColumnarToRow : : :- CometBroadcastHashJoin : : : :- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : : +- Subquery - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometSubqueryBroadcast + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : : +- Subquery + : : : : : +- CometNativeColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometBroadcastHashJoin @@ -84,12 +80,11 @@ CometNativeColumnarToRow : : : : : +- CometBroadcastHashJoin : : : : : :- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometNativeColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : +- CometSubqueryBroadcast + : : : : : : +- CometBroadcastExchange + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : : +- CometBroadcastExchange : : : : : +- CometBroadcastHashJoin : : : : : :- CometFilter @@ -101,12 +96,11 @@ CometNativeColumnarToRow : : : : : : +- CometBroadcastHashJoin : : : : : : :- CometFilter : : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : : : : +- SubqueryBroadcast - : : : : : : : +- BroadcastExchange - : : : : : : : +- CometNativeColumnarToRow - : : : : : : : +- CometProject - : : : : : : : +- CometFilter - : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : : +- CometSubqueryBroadcast + : : : : : : : +- CometBroadcastExchange + : : : : : : : +- CometProject + : : : : : : : +- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : : : +- CometBroadcastExchange : : : : : : +- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.item @@ -125,12 +119,11 @@ CometNativeColumnarToRow : : : : +- CometBroadcastHashJoin : : : : :- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- CometSubqueryBroadcast + : : : : : +- CometBroadcastExchange + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : +- CometBroadcastExchange : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.item @@ -158,12 +151,11 @@ CometNativeColumnarToRow : : : : +- CometBroadcastHashJoin : : : : :- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- CometSubqueryBroadcast + : : : : : +- CometBroadcastExchange + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : +- CometBroadcastExchange : : : : +- CometBroadcastHashJoin : : : : :- CometFilter @@ -175,12 +167,11 @@ CometNativeColumnarToRow : : : : : +- CometBroadcastHashJoin : : : : : :- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometNativeColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : +- CometSubqueryBroadcast + : : : : : : +- CometBroadcastExchange + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : : +- CometBroadcastExchange : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.item @@ -199,12 +190,11 @@ CometNativeColumnarToRow : : : +- CometBroadcastHashJoin : : : :- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometSubqueryBroadcast + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- CometBroadcastExchange : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.item @@ -215,7 +205,11 @@ CometNativeColumnarToRow : +- CometBroadcastExchange : +- CometProject : +- CometFilter - : : +- ReusedSubquery + : : +- Subquery + : : +- CometNativeColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim : +- CometNativeScan parquet spark_catalog.default.date_dim +- CometBroadcastExchange +- CometFilter @@ -230,17 +224,16 @@ CometNativeColumnarToRow : :- CometBroadcastHashJoin : : :- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : : +- Subquery - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometSubqueryBroadcast + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : : +- Subquery + : : : : +- CometNativeColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- CometBroadcastExchange : : +- CometProject : : +- CometBroadcastHashJoin @@ -257,12 +250,11 @@ CometNativeColumnarToRow : : : : +- CometBroadcastHashJoin : : : : :- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- CometSubqueryBroadcast + : : : : : +- CometBroadcastExchange + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : +- CometBroadcastExchange : : : : +- CometBroadcastHashJoin : : : : :- CometFilter @@ -274,12 +266,11 @@ CometNativeColumnarToRow : : : : : +- CometBroadcastHashJoin : : : : : :- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometNativeColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : +- CometSubqueryBroadcast + : : : : : : +- CometBroadcastExchange + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : : +- CometBroadcastExchange : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.item @@ -298,12 +289,11 @@ CometNativeColumnarToRow : : : +- CometBroadcastHashJoin : : : :- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometSubqueryBroadcast + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- CometBroadcastExchange : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.item @@ -331,12 +321,11 @@ CometNativeColumnarToRow : : : +- CometBroadcastHashJoin : : : :- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometSubqueryBroadcast + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- CometBroadcastExchange : : : +- CometBroadcastHashJoin : : : :- CometFilter @@ -348,12 +337,11 @@ CometNativeColumnarToRow : : : : +- CometBroadcastHashJoin : : : : :- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- CometSubqueryBroadcast + : : : : : +- CometBroadcastExchange + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : +- CometBroadcastExchange : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.item @@ -372,12 +360,11 @@ CometNativeColumnarToRow : : +- CometBroadcastHashJoin : : :- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometSubqueryBroadcast + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- CometBroadcastExchange : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.item @@ -388,7 +375,11 @@ CometNativeColumnarToRow +- CometBroadcastExchange +- CometProject +- CometFilter - : +- ReusedSubquery + : +- Subquery + : +- CometNativeColumnarToRow + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.date_dim +- CometNativeScan parquet spark_catalog.default.date_dim -Comet accelerated 331 out of 371 eligible operators (89%). Final plan contains 21 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 354 out of 377 eligible operators (93%). Final plan contains 6 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14a.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14a.native_datafusion/extended.txt index 636e5ce377..83b545c34c 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14a.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14a.native_datafusion/extended.txt @@ -18,12 +18,11 @@ CometNativeColumnarToRow : : : :- CometProject : : : : +- CometBroadcastHashJoin : : : : :- CometNativeScan parquet spark_catalog.default.store_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- CometSubqueryBroadcast + : : : : : +- CometBroadcastExchange + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : +- CometBroadcastExchange : : : : +- CometProject : : : : +- CometFilter @@ -31,12 +30,11 @@ CometNativeColumnarToRow : : : :- CometProject : : : : +- CometBroadcastHashJoin : : : : :- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- CometSubqueryBroadcast + : : : : : +- CometBroadcastExchange + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : +- CometBroadcastExchange : : : : +- CometProject : : : : +- CometFilter @@ -44,12 +42,11 @@ CometNativeColumnarToRow : : : +- CometProject : : : +- CometBroadcastHashJoin : : : :- CometNativeScan parquet spark_catalog.default.web_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometSubqueryBroadcast + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter @@ -64,12 +61,11 @@ CometNativeColumnarToRow : : : :- CometBroadcastHashJoin : : : : :- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- CometSubqueryBroadcast + : : : : : +- CometBroadcastExchange + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : +- CometBroadcastExchange : : : : +- CometProject : : : : +- CometBroadcastHashJoin @@ -86,12 +82,11 @@ CometNativeColumnarToRow : : : : : : +- CometBroadcastHashJoin : : : : : : :- CometFilter : : : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : : : : : +- SubqueryBroadcast - : : : : : : : +- BroadcastExchange - : : : : : : : +- CometNativeColumnarToRow - : : : : : : : +- CometProject - : : : : : : : +- CometFilter - : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : : +- CometSubqueryBroadcast + : : : : : : : +- CometBroadcastExchange + : : : : : : : +- CometProject + : : : : : : : +- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : : : +- CometBroadcastExchange : : : : : : +- CometBroadcastHashJoin : : : : : : :- CometFilter @@ -103,12 +98,11 @@ CometNativeColumnarToRow : : : : : : : +- CometBroadcastHashJoin : : : : : : : :- CometFilter : : : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : : : : : +- SubqueryBroadcast - : : : : : : : : +- BroadcastExchange - : : : : : : : : +- CometNativeColumnarToRow - : : : : : : : : +- CometProject - : : : : : : : : +- CometFilter - : : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : : : +- CometSubqueryBroadcast + : : : : : : : : +- CometBroadcastExchange + : : : : : : : : +- CometProject + : : : : : : : : +- CometFilter + : : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : : : : +- CometBroadcastExchange : : : : : : : +- CometFilter : : : : : : : +- CometNativeScan parquet spark_catalog.default.item @@ -127,12 +121,11 @@ CometNativeColumnarToRow : : : : : +- CometBroadcastHashJoin : : : : : :- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometNativeColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : +- CometSubqueryBroadcast + : : : : : : +- CometBroadcastExchange + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : : +- CometBroadcastExchange : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.item @@ -160,12 +153,11 @@ CometNativeColumnarToRow : : : : : +- CometBroadcastHashJoin : : : : : :- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometNativeColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : +- CometSubqueryBroadcast + : : : : : : +- CometBroadcastExchange + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : : +- CometBroadcastExchange : : : : : +- CometBroadcastHashJoin : : : : : :- CometFilter @@ -177,12 +169,11 @@ CometNativeColumnarToRow : : : : : : +- CometBroadcastHashJoin : : : : : : :- CometFilter : : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : : : : +- SubqueryBroadcast - : : : : : : : +- BroadcastExchange - : : : : : : : +- CometNativeColumnarToRow - : : : : : : : +- CometProject - : : : : : : : +- CometFilter - : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : : +- CometSubqueryBroadcast + : : : : : : : +- CometBroadcastExchange + : : : : : : : +- CometProject + : : : : : : : +- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : : : +- CometBroadcastExchange : : : : : : +- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.item @@ -201,12 +192,11 @@ CometNativeColumnarToRow : : : : +- CometBroadcastHashJoin : : : : :- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- CometSubqueryBroadcast + : : : : : +- CometBroadcastExchange + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : +- CometBroadcastExchange : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.item @@ -230,12 +220,11 @@ CometNativeColumnarToRow : : : :- CometBroadcastHashJoin : : : : :- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- CometSubqueryBroadcast + : : : : : +- CometBroadcastExchange + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : +- CometBroadcastExchange : : : : +- CometProject : : : : +- CometBroadcastHashJoin @@ -252,12 +241,11 @@ CometNativeColumnarToRow : : : : : : +- CometBroadcastHashJoin : : : : : : :- CometFilter : : : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : : : : : +- SubqueryBroadcast - : : : : : : : +- BroadcastExchange - : : : : : : : +- CometNativeColumnarToRow - : : : : : : : +- CometProject - : : : : : : : +- CometFilter - : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : : +- CometSubqueryBroadcast + : : : : : : : +- CometBroadcastExchange + : : : : : : : +- CometProject + : : : : : : : +- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : : : +- CometBroadcastExchange : : : : : : +- CometBroadcastHashJoin : : : : : : :- CometFilter @@ -269,12 +257,11 @@ CometNativeColumnarToRow : : : : : : : +- CometBroadcastHashJoin : : : : : : : :- CometFilter : : : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : : : : : +- SubqueryBroadcast - : : : : : : : : +- BroadcastExchange - : : : : : : : : +- CometNativeColumnarToRow - : : : : : : : : +- CometProject - : : : : : : : : +- CometFilter - : : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : : : +- CometSubqueryBroadcast + : : : : : : : : +- CometBroadcastExchange + : : : : : : : : +- CometProject + : : : : : : : : +- CometFilter + : : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : : : : +- CometBroadcastExchange : : : : : : : +- CometFilter : : : : : : : +- CometNativeScan parquet spark_catalog.default.item @@ -293,12 +280,11 @@ CometNativeColumnarToRow : : : : : +- CometBroadcastHashJoin : : : : : :- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometNativeColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : +- CometSubqueryBroadcast + : : : : : : +- CometBroadcastExchange + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : : +- CometBroadcastExchange : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.item @@ -326,12 +312,11 @@ CometNativeColumnarToRow : : : : : +- CometBroadcastHashJoin : : : : : :- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometNativeColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : +- CometSubqueryBroadcast + : : : : : : +- CometBroadcastExchange + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : : +- CometBroadcastExchange : : : : : +- CometBroadcastHashJoin : : : : : :- CometFilter @@ -343,12 +328,11 @@ CometNativeColumnarToRow : : : : : : +- CometBroadcastHashJoin : : : : : : :- CometFilter : : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : : : : +- SubqueryBroadcast - : : : : : : : +- BroadcastExchange - : : : : : : : +- CometNativeColumnarToRow - : : : : : : : +- CometProject - : : : : : : : +- CometFilter - : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : : +- CometSubqueryBroadcast + : : : : : : : +- CometBroadcastExchange + : : : : : : : +- CometProject + : : : : : : : +- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : : : +- CometBroadcastExchange : : : : : : +- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.item @@ -367,12 +351,11 @@ CometNativeColumnarToRow : : : : +- CometBroadcastHashJoin : : : : :- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- CometSubqueryBroadcast + : : : : : +- CometBroadcastExchange + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : +- CometBroadcastExchange : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.item @@ -396,12 +379,11 @@ CometNativeColumnarToRow : : :- CometBroadcastHashJoin : : : :- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometSubqueryBroadcast + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometBroadcastHashJoin @@ -418,12 +400,11 @@ CometNativeColumnarToRow : : : : : +- CometBroadcastHashJoin : : : : : :- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometNativeColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : +- CometSubqueryBroadcast + : : : : : : +- CometBroadcastExchange + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : : +- CometBroadcastExchange : : : : : +- CometBroadcastHashJoin : : : : : :- CometFilter @@ -435,12 +416,11 @@ CometNativeColumnarToRow : : : : : : +- CometBroadcastHashJoin : : : : : : :- CometFilter : : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : : : : +- SubqueryBroadcast - : : : : : : : +- BroadcastExchange - : : : : : : : +- CometNativeColumnarToRow - : : : : : : : +- CometProject - : : : : : : : +- CometFilter - : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : : +- CometSubqueryBroadcast + : : : : : : : +- CometBroadcastExchange + : : : : : : : +- CometProject + : : : : : : : +- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : : : +- CometBroadcastExchange : : : : : : +- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.item @@ -459,12 +439,11 @@ CometNativeColumnarToRow : : : : +- CometBroadcastHashJoin : : : : :- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- CometSubqueryBroadcast + : : : : : +- CometBroadcastExchange + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : +- CometBroadcastExchange : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.item @@ -492,12 +471,11 @@ CometNativeColumnarToRow : : : : +- CometBroadcastHashJoin : : : : :- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- CometSubqueryBroadcast + : : : : : +- CometBroadcastExchange + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : +- CometBroadcastExchange : : : : +- CometBroadcastHashJoin : : : : :- CometFilter @@ -509,12 +487,11 @@ CometNativeColumnarToRow : : : : : +- CometBroadcastHashJoin : : : : : :- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometNativeColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : +- CometSubqueryBroadcast + : : : : : : +- CometBroadcastExchange + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : : +- CometBroadcastExchange : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.item @@ -533,12 +510,11 @@ CometNativeColumnarToRow : : : +- CometBroadcastHashJoin : : : :- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometSubqueryBroadcast + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- CometBroadcastExchange : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.item @@ -567,12 +543,11 @@ CometNativeColumnarToRow : : : :- CometProject : : : : +- CometBroadcastHashJoin : : : : :- CometNativeScan parquet spark_catalog.default.store_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- CometSubqueryBroadcast + : : : : : +- CometBroadcastExchange + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : +- CometBroadcastExchange : : : : +- CometProject : : : : +- CometFilter @@ -580,12 +555,11 @@ CometNativeColumnarToRow : : : :- CometProject : : : : +- CometBroadcastHashJoin : : : : :- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- CometSubqueryBroadcast + : : : : : +- CometBroadcastExchange + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : +- CometBroadcastExchange : : : : +- CometProject : : : : +- CometFilter @@ -593,12 +567,11 @@ CometNativeColumnarToRow : : : +- CometProject : : : +- CometBroadcastHashJoin : : : :- CometNativeScan parquet spark_catalog.default.web_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometSubqueryBroadcast + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter @@ -613,12 +586,11 @@ CometNativeColumnarToRow : : : :- CometBroadcastHashJoin : : : : :- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- CometSubqueryBroadcast + : : : : : +- CometBroadcastExchange + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : +- CometBroadcastExchange : : : : +- CometProject : : : : +- CometBroadcastHashJoin @@ -635,12 +607,11 @@ CometNativeColumnarToRow : : : : : : +- CometBroadcastHashJoin : : : : : : :- CometFilter : : : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : : : : : +- SubqueryBroadcast - : : : : : : : +- BroadcastExchange - : : : : : : : +- CometNativeColumnarToRow - : : : : : : : +- CometProject - : : : : : : : +- CometFilter - : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : : +- CometSubqueryBroadcast + : : : : : : : +- CometBroadcastExchange + : : : : : : : +- CometProject + : : : : : : : +- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : : : +- CometBroadcastExchange : : : : : : +- CometBroadcastHashJoin : : : : : : :- CometFilter @@ -652,12 +623,11 @@ CometNativeColumnarToRow : : : : : : : +- CometBroadcastHashJoin : : : : : : : :- CometFilter : : : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : : : : : +- SubqueryBroadcast - : : : : : : : : +- BroadcastExchange - : : : : : : : : +- CometNativeColumnarToRow - : : : : : : : : +- CometProject - : : : : : : : : +- CometFilter - : : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : : : +- CometSubqueryBroadcast + : : : : : : : : +- CometBroadcastExchange + : : : : : : : : +- CometProject + : : : : : : : : +- CometFilter + : : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : : : : +- CometBroadcastExchange : : : : : : : +- CometFilter : : : : : : : +- CometNativeScan parquet spark_catalog.default.item @@ -676,12 +646,11 @@ CometNativeColumnarToRow : : : : : +- CometBroadcastHashJoin : : : : : :- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometNativeColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : +- CometSubqueryBroadcast + : : : : : : +- CometBroadcastExchange + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : : +- CometBroadcastExchange : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.item @@ -709,12 +678,11 @@ CometNativeColumnarToRow : : : : : +- CometBroadcastHashJoin : : : : : :- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometNativeColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : +- CometSubqueryBroadcast + : : : : : : +- CometBroadcastExchange + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : : +- CometBroadcastExchange : : : : : +- CometBroadcastHashJoin : : : : : :- CometFilter @@ -726,12 +694,11 @@ CometNativeColumnarToRow : : : : : : +- CometBroadcastHashJoin : : : : : : :- CometFilter : : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : : : : +- SubqueryBroadcast - : : : : : : : +- BroadcastExchange - : : : : : : : +- CometNativeColumnarToRow - : : : : : : : +- CometProject - : : : : : : : +- CometFilter - : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : : +- CometSubqueryBroadcast + : : : : : : : +- CometBroadcastExchange + : : : : : : : +- CometProject + : : : : : : : +- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : : : +- CometBroadcastExchange : : : : : : +- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.item @@ -750,12 +717,11 @@ CometNativeColumnarToRow : : : : +- CometBroadcastHashJoin : : : : :- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- CometSubqueryBroadcast + : : : : : +- CometBroadcastExchange + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : +- CometBroadcastExchange : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.item @@ -779,12 +745,11 @@ CometNativeColumnarToRow : : : :- CometBroadcastHashJoin : : : : :- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- CometSubqueryBroadcast + : : : : : +- CometBroadcastExchange + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : +- CometBroadcastExchange : : : : +- CometProject : : : : +- CometBroadcastHashJoin @@ -801,12 +766,11 @@ CometNativeColumnarToRow : : : : : : +- CometBroadcastHashJoin : : : : : : :- CometFilter : : : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : : : : : +- SubqueryBroadcast - : : : : : : : +- BroadcastExchange - : : : : : : : +- CometNativeColumnarToRow - : : : : : : : +- CometProject - : : : : : : : +- CometFilter - : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : : +- CometSubqueryBroadcast + : : : : : : : +- CometBroadcastExchange + : : : : : : : +- CometProject + : : : : : : : +- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : : : +- CometBroadcastExchange : : : : : : +- CometBroadcastHashJoin : : : : : : :- CometFilter @@ -818,12 +782,11 @@ CometNativeColumnarToRow : : : : : : : +- CometBroadcastHashJoin : : : : : : : :- CometFilter : : : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : : : : : +- SubqueryBroadcast - : : : : : : : : +- BroadcastExchange - : : : : : : : : +- CometNativeColumnarToRow - : : : : : : : : +- CometProject - : : : : : : : : +- CometFilter - : : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : : : +- CometSubqueryBroadcast + : : : : : : : : +- CometBroadcastExchange + : : : : : : : : +- CometProject + : : : : : : : : +- CometFilter + : : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : : : : +- CometBroadcastExchange : : : : : : : +- CometFilter : : : : : : : +- CometNativeScan parquet spark_catalog.default.item @@ -842,12 +805,11 @@ CometNativeColumnarToRow : : : : : +- CometBroadcastHashJoin : : : : : :- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometNativeColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : +- CometSubqueryBroadcast + : : : : : : +- CometBroadcastExchange + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : : +- CometBroadcastExchange : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.item @@ -875,12 +837,11 @@ CometNativeColumnarToRow : : : : : +- CometBroadcastHashJoin : : : : : :- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometNativeColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : +- CometSubqueryBroadcast + : : : : : : +- CometBroadcastExchange + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : : +- CometBroadcastExchange : : : : : +- CometBroadcastHashJoin : : : : : :- CometFilter @@ -892,12 +853,11 @@ CometNativeColumnarToRow : : : : : : +- CometBroadcastHashJoin : : : : : : :- CometFilter : : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : : : : +- SubqueryBroadcast - : : : : : : : +- BroadcastExchange - : : : : : : : +- CometNativeColumnarToRow - : : : : : : : +- CometProject - : : : : : : : +- CometFilter - : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : : +- CometSubqueryBroadcast + : : : : : : : +- CometBroadcastExchange + : : : : : : : +- CometProject + : : : : : : : +- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : : : +- CometBroadcastExchange : : : : : : +- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.item @@ -916,12 +876,11 @@ CometNativeColumnarToRow : : : : +- CometBroadcastHashJoin : : : : :- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- CometSubqueryBroadcast + : : : : : +- CometBroadcastExchange + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : +- CometBroadcastExchange : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.item @@ -945,12 +904,11 @@ CometNativeColumnarToRow : : :- CometBroadcastHashJoin : : : :- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometSubqueryBroadcast + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometBroadcastHashJoin @@ -967,12 +925,11 @@ CometNativeColumnarToRow : : : : : +- CometBroadcastHashJoin : : : : : :- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometNativeColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : +- CometSubqueryBroadcast + : : : : : : +- CometBroadcastExchange + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : : +- CometBroadcastExchange : : : : : +- CometBroadcastHashJoin : : : : : :- CometFilter @@ -984,12 +941,11 @@ CometNativeColumnarToRow : : : : : : +- CometBroadcastHashJoin : : : : : : :- CometFilter : : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : : : : +- SubqueryBroadcast - : : : : : : : +- BroadcastExchange - : : : : : : : +- CometNativeColumnarToRow - : : : : : : : +- CometProject - : : : : : : : +- CometFilter - : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : : +- CometSubqueryBroadcast + : : : : : : : +- CometBroadcastExchange + : : : : : : : +- CometProject + : : : : : : : +- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : : : +- CometBroadcastExchange : : : : : : +- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.item @@ -1008,12 +964,11 @@ CometNativeColumnarToRow : : : : +- CometBroadcastHashJoin : : : : :- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- CometSubqueryBroadcast + : : : : : +- CometBroadcastExchange + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : +- CometBroadcastExchange : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.item @@ -1041,12 +996,11 @@ CometNativeColumnarToRow : : : : +- CometBroadcastHashJoin : : : : :- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- CometSubqueryBroadcast + : : : : : +- CometBroadcastExchange + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : +- CometBroadcastExchange : : : : +- CometBroadcastHashJoin : : : : :- CometFilter @@ -1058,12 +1012,11 @@ CometNativeColumnarToRow : : : : : +- CometBroadcastHashJoin : : : : : :- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometNativeColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : +- CometSubqueryBroadcast + : : : : : : +- CometBroadcastExchange + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : : +- CometBroadcastExchange : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.item @@ -1082,12 +1035,11 @@ CometNativeColumnarToRow : : : +- CometBroadcastHashJoin : : : :- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometSubqueryBroadcast + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- CometBroadcastExchange : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.item @@ -1116,12 +1068,11 @@ CometNativeColumnarToRow : : : :- CometProject : : : : +- CometBroadcastHashJoin : : : : :- CometNativeScan parquet spark_catalog.default.store_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- CometSubqueryBroadcast + : : : : : +- CometBroadcastExchange + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : +- CometBroadcastExchange : : : : +- CometProject : : : : +- CometFilter @@ -1129,12 +1080,11 @@ CometNativeColumnarToRow : : : :- CometProject : : : : +- CometBroadcastHashJoin : : : : :- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- CometSubqueryBroadcast + : : : : : +- CometBroadcastExchange + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : +- CometBroadcastExchange : : : : +- CometProject : : : : +- CometFilter @@ -1142,12 +1092,11 @@ CometNativeColumnarToRow : : : +- CometProject : : : +- CometBroadcastHashJoin : : : :- CometNativeScan parquet spark_catalog.default.web_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometSubqueryBroadcast + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter @@ -1162,12 +1111,11 @@ CometNativeColumnarToRow : : : :- CometBroadcastHashJoin : : : : :- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- CometSubqueryBroadcast + : : : : : +- CometBroadcastExchange + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : +- CometBroadcastExchange : : : : +- CometProject : : : : +- CometBroadcastHashJoin @@ -1184,12 +1132,11 @@ CometNativeColumnarToRow : : : : : : +- CometBroadcastHashJoin : : : : : : :- CometFilter : : : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : : : : : +- SubqueryBroadcast - : : : : : : : +- BroadcastExchange - : : : : : : : +- CometNativeColumnarToRow - : : : : : : : +- CometProject - : : : : : : : +- CometFilter - : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : : +- CometSubqueryBroadcast + : : : : : : : +- CometBroadcastExchange + : : : : : : : +- CometProject + : : : : : : : +- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : : : +- CometBroadcastExchange : : : : : : +- CometBroadcastHashJoin : : : : : : :- CometFilter @@ -1201,12 +1148,11 @@ CometNativeColumnarToRow : : : : : : : +- CometBroadcastHashJoin : : : : : : : :- CometFilter : : : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : : : : : +- SubqueryBroadcast - : : : : : : : : +- BroadcastExchange - : : : : : : : : +- CometNativeColumnarToRow - : : : : : : : : +- CometProject - : : : : : : : : +- CometFilter - : : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : : : +- CometSubqueryBroadcast + : : : : : : : : +- CometBroadcastExchange + : : : : : : : : +- CometProject + : : : : : : : : +- CometFilter + : : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : : : : +- CometBroadcastExchange : : : : : : : +- CometFilter : : : : : : : +- CometNativeScan parquet spark_catalog.default.item @@ -1225,12 +1171,11 @@ CometNativeColumnarToRow : : : : : +- CometBroadcastHashJoin : : : : : :- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometNativeColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : +- CometSubqueryBroadcast + : : : : : : +- CometBroadcastExchange + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : : +- CometBroadcastExchange : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.item @@ -1258,12 +1203,11 @@ CometNativeColumnarToRow : : : : : +- CometBroadcastHashJoin : : : : : :- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometNativeColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : +- CometSubqueryBroadcast + : : : : : : +- CometBroadcastExchange + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : : +- CometBroadcastExchange : : : : : +- CometBroadcastHashJoin : : : : : :- CometFilter @@ -1275,12 +1219,11 @@ CometNativeColumnarToRow : : : : : : +- CometBroadcastHashJoin : : : : : : :- CometFilter : : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : : : : +- SubqueryBroadcast - : : : : : : : +- BroadcastExchange - : : : : : : : +- CometNativeColumnarToRow - : : : : : : : +- CometProject - : : : : : : : +- CometFilter - : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : : +- CometSubqueryBroadcast + : : : : : : : +- CometBroadcastExchange + : : : : : : : +- CometProject + : : : : : : : +- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : : : +- CometBroadcastExchange : : : : : : +- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.item @@ -1299,12 +1242,11 @@ CometNativeColumnarToRow : : : : +- CometBroadcastHashJoin : : : : :- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- CometSubqueryBroadcast + : : : : : +- CometBroadcastExchange + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : +- CometBroadcastExchange : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.item @@ -1328,12 +1270,11 @@ CometNativeColumnarToRow : : : :- CometBroadcastHashJoin : : : : :- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- CometSubqueryBroadcast + : : : : : +- CometBroadcastExchange + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : +- CometBroadcastExchange : : : : +- CometProject : : : : +- CometBroadcastHashJoin @@ -1350,12 +1291,11 @@ CometNativeColumnarToRow : : : : : : +- CometBroadcastHashJoin : : : : : : :- CometFilter : : : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : : : : : +- SubqueryBroadcast - : : : : : : : +- BroadcastExchange - : : : : : : : +- CometNativeColumnarToRow - : : : : : : : +- CometProject - : : : : : : : +- CometFilter - : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : : +- CometSubqueryBroadcast + : : : : : : : +- CometBroadcastExchange + : : : : : : : +- CometProject + : : : : : : : +- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : : : +- CometBroadcastExchange : : : : : : +- CometBroadcastHashJoin : : : : : : :- CometFilter @@ -1367,12 +1307,11 @@ CometNativeColumnarToRow : : : : : : : +- CometBroadcastHashJoin : : : : : : : :- CometFilter : : : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : : : : : +- SubqueryBroadcast - : : : : : : : : +- BroadcastExchange - : : : : : : : : +- CometNativeColumnarToRow - : : : : : : : : +- CometProject - : : : : : : : : +- CometFilter - : : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : : : +- CometSubqueryBroadcast + : : : : : : : : +- CometBroadcastExchange + : : : : : : : : +- CometProject + : : : : : : : : +- CometFilter + : : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : : : : +- CometBroadcastExchange : : : : : : : +- CometFilter : : : : : : : +- CometNativeScan parquet spark_catalog.default.item @@ -1391,12 +1330,11 @@ CometNativeColumnarToRow : : : : : +- CometBroadcastHashJoin : : : : : :- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometNativeColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : +- CometSubqueryBroadcast + : : : : : : +- CometBroadcastExchange + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : : +- CometBroadcastExchange : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.item @@ -1424,12 +1362,11 @@ CometNativeColumnarToRow : : : : : +- CometBroadcastHashJoin : : : : : :- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometNativeColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : +- CometSubqueryBroadcast + : : : : : : +- CometBroadcastExchange + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : : +- CometBroadcastExchange : : : : : +- CometBroadcastHashJoin : : : : : :- CometFilter @@ -1441,12 +1378,11 @@ CometNativeColumnarToRow : : : : : : +- CometBroadcastHashJoin : : : : : : :- CometFilter : : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : : : : +- SubqueryBroadcast - : : : : : : : +- BroadcastExchange - : : : : : : : +- CometNativeColumnarToRow - : : : : : : : +- CometProject - : : : : : : : +- CometFilter - : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : : +- CometSubqueryBroadcast + : : : : : : : +- CometBroadcastExchange + : : : : : : : +- CometProject + : : : : : : : +- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : : : +- CometBroadcastExchange : : : : : : +- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.item @@ -1465,12 +1401,11 @@ CometNativeColumnarToRow : : : : +- CometBroadcastHashJoin : : : : :- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- CometSubqueryBroadcast + : : : : : +- CometBroadcastExchange + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : +- CometBroadcastExchange : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.item @@ -1494,12 +1429,11 @@ CometNativeColumnarToRow : : :- CometBroadcastHashJoin : : : :- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometSubqueryBroadcast + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometBroadcastHashJoin @@ -1516,12 +1450,11 @@ CometNativeColumnarToRow : : : : : +- CometBroadcastHashJoin : : : : : :- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometNativeColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : +- CometSubqueryBroadcast + : : : : : : +- CometBroadcastExchange + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : : +- CometBroadcastExchange : : : : : +- CometBroadcastHashJoin : : : : : :- CometFilter @@ -1533,12 +1466,11 @@ CometNativeColumnarToRow : : : : : : +- CometBroadcastHashJoin : : : : : : :- CometFilter : : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : : : : +- SubqueryBroadcast - : : : : : : : +- BroadcastExchange - : : : : : : : +- CometNativeColumnarToRow - : : : : : : : +- CometProject - : : : : : : : +- CometFilter - : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : : +- CometSubqueryBroadcast + : : : : : : : +- CometBroadcastExchange + : : : : : : : +- CometProject + : : : : : : : +- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : : : +- CometBroadcastExchange : : : : : : +- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.item @@ -1557,12 +1489,11 @@ CometNativeColumnarToRow : : : : +- CometBroadcastHashJoin : : : : :- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- CometSubqueryBroadcast + : : : : : +- CometBroadcastExchange + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : +- CometBroadcastExchange : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.item @@ -1590,12 +1521,11 @@ CometNativeColumnarToRow : : : : +- CometBroadcastHashJoin : : : : :- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- CometSubqueryBroadcast + : : : : : +- CometBroadcastExchange + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : +- CometBroadcastExchange : : : : +- CometBroadcastHashJoin : : : : :- CometFilter @@ -1607,12 +1537,11 @@ CometNativeColumnarToRow : : : : : +- CometBroadcastHashJoin : : : : : :- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometNativeColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : +- CometSubqueryBroadcast + : : : : : : +- CometBroadcastExchange + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : : +- CometBroadcastExchange : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.item @@ -1631,12 +1560,11 @@ CometNativeColumnarToRow : : : +- CometBroadcastHashJoin : : : :- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometSubqueryBroadcast + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- CometBroadcastExchange : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.item @@ -1665,12 +1593,11 @@ CometNativeColumnarToRow : : : :- CometProject : : : : +- CometBroadcastHashJoin : : : : :- CometNativeScan parquet spark_catalog.default.store_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- CometSubqueryBroadcast + : : : : : +- CometBroadcastExchange + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : +- CometBroadcastExchange : : : : +- CometProject : : : : +- CometFilter @@ -1678,12 +1605,11 @@ CometNativeColumnarToRow : : : :- CometProject : : : : +- CometBroadcastHashJoin : : : : :- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- CometSubqueryBroadcast + : : : : : +- CometBroadcastExchange + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : +- CometBroadcastExchange : : : : +- CometProject : : : : +- CometFilter @@ -1691,12 +1617,11 @@ CometNativeColumnarToRow : : : +- CometProject : : : +- CometBroadcastHashJoin : : : :- CometNativeScan parquet spark_catalog.default.web_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometSubqueryBroadcast + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter @@ -1711,12 +1636,11 @@ CometNativeColumnarToRow : : : :- CometBroadcastHashJoin : : : : :- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- CometSubqueryBroadcast + : : : : : +- CometBroadcastExchange + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : +- CometBroadcastExchange : : : : +- CometProject : : : : +- CometBroadcastHashJoin @@ -1733,12 +1657,11 @@ CometNativeColumnarToRow : : : : : : +- CometBroadcastHashJoin : : : : : : :- CometFilter : : : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : : : : : +- SubqueryBroadcast - : : : : : : : +- BroadcastExchange - : : : : : : : +- CometNativeColumnarToRow - : : : : : : : +- CometProject - : : : : : : : +- CometFilter - : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : : +- CometSubqueryBroadcast + : : : : : : : +- CometBroadcastExchange + : : : : : : : +- CometProject + : : : : : : : +- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : : : +- CometBroadcastExchange : : : : : : +- CometBroadcastHashJoin : : : : : : :- CometFilter @@ -1750,12 +1673,11 @@ CometNativeColumnarToRow : : : : : : : +- CometBroadcastHashJoin : : : : : : : :- CometFilter : : : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : : : : : +- SubqueryBroadcast - : : : : : : : : +- BroadcastExchange - : : : : : : : : +- CometNativeColumnarToRow - : : : : : : : : +- CometProject - : : : : : : : : +- CometFilter - : : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : : : +- CometSubqueryBroadcast + : : : : : : : : +- CometBroadcastExchange + : : : : : : : : +- CometProject + : : : : : : : : +- CometFilter + : : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : : : : +- CometBroadcastExchange : : : : : : : +- CometFilter : : : : : : : +- CometNativeScan parquet spark_catalog.default.item @@ -1774,12 +1696,11 @@ CometNativeColumnarToRow : : : : : +- CometBroadcastHashJoin : : : : : :- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometNativeColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : +- CometSubqueryBroadcast + : : : : : : +- CometBroadcastExchange + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : : +- CometBroadcastExchange : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.item @@ -1807,12 +1728,11 @@ CometNativeColumnarToRow : : : : : +- CometBroadcastHashJoin : : : : : :- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometNativeColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : +- CometSubqueryBroadcast + : : : : : : +- CometBroadcastExchange + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : : +- CometBroadcastExchange : : : : : +- CometBroadcastHashJoin : : : : : :- CometFilter @@ -1824,12 +1744,11 @@ CometNativeColumnarToRow : : : : : : +- CometBroadcastHashJoin : : : : : : :- CometFilter : : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : : : : +- SubqueryBroadcast - : : : : : : : +- BroadcastExchange - : : : : : : : +- CometNativeColumnarToRow - : : : : : : : +- CometProject - : : : : : : : +- CometFilter - : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : : +- CometSubqueryBroadcast + : : : : : : : +- CometBroadcastExchange + : : : : : : : +- CometProject + : : : : : : : +- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : : : +- CometBroadcastExchange : : : : : : +- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.item @@ -1848,12 +1767,11 @@ CometNativeColumnarToRow : : : : +- CometBroadcastHashJoin : : : : :- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- CometSubqueryBroadcast + : : : : : +- CometBroadcastExchange + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : +- CometBroadcastExchange : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.item @@ -1877,12 +1795,11 @@ CometNativeColumnarToRow : : : :- CometBroadcastHashJoin : : : : :- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- CometSubqueryBroadcast + : : : : : +- CometBroadcastExchange + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : +- CometBroadcastExchange : : : : +- CometProject : : : : +- CometBroadcastHashJoin @@ -1899,12 +1816,11 @@ CometNativeColumnarToRow : : : : : : +- CometBroadcastHashJoin : : : : : : :- CometFilter : : : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : : : : : +- SubqueryBroadcast - : : : : : : : +- BroadcastExchange - : : : : : : : +- CometNativeColumnarToRow - : : : : : : : +- CometProject - : : : : : : : +- CometFilter - : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : : +- CometSubqueryBroadcast + : : : : : : : +- CometBroadcastExchange + : : : : : : : +- CometProject + : : : : : : : +- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : : : +- CometBroadcastExchange : : : : : : +- CometBroadcastHashJoin : : : : : : :- CometFilter @@ -1916,12 +1832,11 @@ CometNativeColumnarToRow : : : : : : : +- CometBroadcastHashJoin : : : : : : : :- CometFilter : : : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : : : : : +- SubqueryBroadcast - : : : : : : : : +- BroadcastExchange - : : : : : : : : +- CometNativeColumnarToRow - : : : : : : : : +- CometProject - : : : : : : : : +- CometFilter - : : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : : : +- CometSubqueryBroadcast + : : : : : : : : +- CometBroadcastExchange + : : : : : : : : +- CometProject + : : : : : : : : +- CometFilter + : : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : : : : +- CometBroadcastExchange : : : : : : : +- CometFilter : : : : : : : +- CometNativeScan parquet spark_catalog.default.item @@ -1940,12 +1855,11 @@ CometNativeColumnarToRow : : : : : +- CometBroadcastHashJoin : : : : : :- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometNativeColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : +- CometSubqueryBroadcast + : : : : : : +- CometBroadcastExchange + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : : +- CometBroadcastExchange : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.item @@ -1973,12 +1887,11 @@ CometNativeColumnarToRow : : : : : +- CometBroadcastHashJoin : : : : : :- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometNativeColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : +- CometSubqueryBroadcast + : : : : : : +- CometBroadcastExchange + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : : +- CometBroadcastExchange : : : : : +- CometBroadcastHashJoin : : : : : :- CometFilter @@ -1990,12 +1903,11 @@ CometNativeColumnarToRow : : : : : : +- CometBroadcastHashJoin : : : : : : :- CometFilter : : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : : : : +- SubqueryBroadcast - : : : : : : : +- BroadcastExchange - : : : : : : : +- CometNativeColumnarToRow - : : : : : : : +- CometProject - : : : : : : : +- CometFilter - : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : : +- CometSubqueryBroadcast + : : : : : : : +- CometBroadcastExchange + : : : : : : : +- CometProject + : : : : : : : +- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : : : +- CometBroadcastExchange : : : : : : +- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.item @@ -2014,12 +1926,11 @@ CometNativeColumnarToRow : : : : +- CometBroadcastHashJoin : : : : :- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- CometSubqueryBroadcast + : : : : : +- CometBroadcastExchange + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : +- CometBroadcastExchange : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.item @@ -2043,12 +1954,11 @@ CometNativeColumnarToRow : : :- CometBroadcastHashJoin : : : :- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometSubqueryBroadcast + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometBroadcastHashJoin @@ -2065,12 +1975,11 @@ CometNativeColumnarToRow : : : : : +- CometBroadcastHashJoin : : : : : :- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometNativeColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : +- CometSubqueryBroadcast + : : : : : : +- CometBroadcastExchange + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : : +- CometBroadcastExchange : : : : : +- CometBroadcastHashJoin : : : : : :- CometFilter @@ -2082,12 +1991,11 @@ CometNativeColumnarToRow : : : : : : +- CometBroadcastHashJoin : : : : : : :- CometFilter : : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : : : : +- SubqueryBroadcast - : : : : : : : +- BroadcastExchange - : : : : : : : +- CometNativeColumnarToRow - : : : : : : : +- CometProject - : : : : : : : +- CometFilter - : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : : +- CometSubqueryBroadcast + : : : : : : : +- CometBroadcastExchange + : : : : : : : +- CometProject + : : : : : : : +- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : : : +- CometBroadcastExchange : : : : : : +- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.item @@ -2106,12 +2014,11 @@ CometNativeColumnarToRow : : : : +- CometBroadcastHashJoin : : : : :- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- CometSubqueryBroadcast + : : : : : +- CometBroadcastExchange + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : +- CometBroadcastExchange : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.item @@ -2139,12 +2046,11 @@ CometNativeColumnarToRow : : : : +- CometBroadcastHashJoin : : : : :- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- CometSubqueryBroadcast + : : : : : +- CometBroadcastExchange + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : +- CometBroadcastExchange : : : : +- CometBroadcastHashJoin : : : : :- CometFilter @@ -2156,12 +2062,11 @@ CometNativeColumnarToRow : : : : : +- CometBroadcastHashJoin : : : : : :- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometNativeColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : +- CometSubqueryBroadcast + : : : : : : +- CometBroadcastExchange + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : : +- CometBroadcastExchange : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.item @@ -2180,12 +2085,11 @@ CometNativeColumnarToRow : : : +- CometBroadcastHashJoin : : : :- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometSubqueryBroadcast + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- CometBroadcastExchange : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.item @@ -2214,12 +2118,11 @@ CometNativeColumnarToRow : : :- CometProject : : : +- CometBroadcastHashJoin : : : :- CometNativeScan parquet spark_catalog.default.store_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometSubqueryBroadcast + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter @@ -2227,12 +2130,11 @@ CometNativeColumnarToRow : : :- CometProject : : : +- CometBroadcastHashJoin : : : :- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometSubqueryBroadcast + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter @@ -2240,12 +2142,11 @@ CometNativeColumnarToRow : : +- CometProject : : +- CometBroadcastHashJoin : : :- CometNativeScan parquet spark_catalog.default.web_sales - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometSubqueryBroadcast + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- CometBroadcastExchange : : +- CometProject : : +- CometFilter @@ -2260,12 +2161,11 @@ CometNativeColumnarToRow : : :- CometBroadcastHashJoin : : : :- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometSubqueryBroadcast + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometBroadcastHashJoin @@ -2282,12 +2182,11 @@ CometNativeColumnarToRow : : : : : +- CometBroadcastHashJoin : : : : : :- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometNativeColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : +- CometSubqueryBroadcast + : : : : : : +- CometBroadcastExchange + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : : +- CometBroadcastExchange : : : : : +- CometBroadcastHashJoin : : : : : :- CometFilter @@ -2299,12 +2198,11 @@ CometNativeColumnarToRow : : : : : : +- CometBroadcastHashJoin : : : : : : :- CometFilter : : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : : : : +- SubqueryBroadcast - : : : : : : : +- BroadcastExchange - : : : : : : : +- CometNativeColumnarToRow - : : : : : : : +- CometProject - : : : : : : : +- CometFilter - : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : : +- CometSubqueryBroadcast + : : : : : : : +- CometBroadcastExchange + : : : : : : : +- CometProject + : : : : : : : +- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : : : +- CometBroadcastExchange : : : : : : +- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.item @@ -2323,12 +2221,11 @@ CometNativeColumnarToRow : : : : +- CometBroadcastHashJoin : : : : :- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- CometSubqueryBroadcast + : : : : : +- CometBroadcastExchange + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : +- CometBroadcastExchange : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.item @@ -2356,12 +2253,11 @@ CometNativeColumnarToRow : : : : +- CometBroadcastHashJoin : : : : :- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- CometSubqueryBroadcast + : : : : : +- CometBroadcastExchange + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : +- CometBroadcastExchange : : : : +- CometBroadcastHashJoin : : : : :- CometFilter @@ -2373,12 +2269,11 @@ CometNativeColumnarToRow : : : : : +- CometBroadcastHashJoin : : : : : :- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometNativeColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : +- CometSubqueryBroadcast + : : : : : : +- CometBroadcastExchange + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : : +- CometBroadcastExchange : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.item @@ -2397,12 +2292,11 @@ CometNativeColumnarToRow : : : +- CometBroadcastHashJoin : : : :- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometSubqueryBroadcast + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- CometBroadcastExchange : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.item @@ -2426,12 +2320,11 @@ CometNativeColumnarToRow : : :- CometBroadcastHashJoin : : : :- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometSubqueryBroadcast + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometBroadcastHashJoin @@ -2448,12 +2341,11 @@ CometNativeColumnarToRow : : : : : +- CometBroadcastHashJoin : : : : : :- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometNativeColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : +- CometSubqueryBroadcast + : : : : : : +- CometBroadcastExchange + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : : +- CometBroadcastExchange : : : : : +- CometBroadcastHashJoin : : : : : :- CometFilter @@ -2465,12 +2357,11 @@ CometNativeColumnarToRow : : : : : : +- CometBroadcastHashJoin : : : : : : :- CometFilter : : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : : : : +- SubqueryBroadcast - : : : : : : : +- BroadcastExchange - : : : : : : : +- CometNativeColumnarToRow - : : : : : : : +- CometProject - : : : : : : : +- CometFilter - : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : : +- CometSubqueryBroadcast + : : : : : : : +- CometBroadcastExchange + : : : : : : : +- CometProject + : : : : : : : +- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : : : +- CometBroadcastExchange : : : : : : +- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.item @@ -2489,12 +2380,11 @@ CometNativeColumnarToRow : : : : +- CometBroadcastHashJoin : : : : :- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- CometSubqueryBroadcast + : : : : : +- CometBroadcastExchange + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : +- CometBroadcastExchange : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.item @@ -2522,12 +2412,11 @@ CometNativeColumnarToRow : : : : +- CometBroadcastHashJoin : : : : :- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- CometSubqueryBroadcast + : : : : : +- CometBroadcastExchange + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : +- CometBroadcastExchange : : : : +- CometBroadcastHashJoin : : : : :- CometFilter @@ -2539,12 +2428,11 @@ CometNativeColumnarToRow : : : : : +- CometBroadcastHashJoin : : : : : :- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometNativeColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : +- CometSubqueryBroadcast + : : : : : : +- CometBroadcastExchange + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : : +- CometBroadcastExchange : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.item @@ -2563,12 +2451,11 @@ CometNativeColumnarToRow : : : +- CometBroadcastHashJoin : : : :- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometSubqueryBroadcast + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- CometBroadcastExchange : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.item @@ -2592,12 +2479,11 @@ CometNativeColumnarToRow : :- CometBroadcastHashJoin : : :- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometSubqueryBroadcast + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- CometBroadcastExchange : : +- CometProject : : +- CometBroadcastHashJoin @@ -2614,12 +2500,11 @@ CometNativeColumnarToRow : : : : +- CometBroadcastHashJoin : : : : :- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- CometSubqueryBroadcast + : : : : : +- CometBroadcastExchange + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : +- CometBroadcastExchange : : : : +- CometBroadcastHashJoin : : : : :- CometFilter @@ -2631,12 +2516,11 @@ CometNativeColumnarToRow : : : : : +- CometBroadcastHashJoin : : : : : :- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometNativeColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : +- CometSubqueryBroadcast + : : : : : : +- CometBroadcastExchange + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : : +- CometBroadcastExchange : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.item @@ -2655,12 +2539,11 @@ CometNativeColumnarToRow : : : +- CometBroadcastHashJoin : : : :- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometSubqueryBroadcast + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- CometBroadcastExchange : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.item @@ -2688,12 +2571,11 @@ CometNativeColumnarToRow : : : +- CometBroadcastHashJoin : : : :- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometSubqueryBroadcast + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- CometBroadcastExchange : : : +- CometBroadcastHashJoin : : : :- CometFilter @@ -2705,12 +2587,11 @@ CometNativeColumnarToRow : : : : +- CometBroadcastHashJoin : : : : :- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- CometSubqueryBroadcast + : : : : : +- CometBroadcastExchange + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : +- CometBroadcastExchange : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.item @@ -2729,12 +2610,11 @@ CometNativeColumnarToRow : : +- CometBroadcastHashJoin : : :- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometSubqueryBroadcast + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- CometBroadcastExchange : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.item @@ -2747,4 +2627,4 @@ CometNativeColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.date_dim -Comet accelerated 2367 out of 2622 eligible operators (90%). Final plan contains 126 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 2487 out of 2622 eligible operators (94%). Final plan contains 6 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q18a.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q18a.native_datafusion/extended.txt index 82bacea2fb..b34ebfdf84 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q18a.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q18a.native_datafusion/extended.txt @@ -18,12 +18,11 @@ CometNativeColumnarToRow : : : : : : +- CometBroadcastHashJoin : : : : : : :- CometFilter : : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : : : : +- SubqueryBroadcast - : : : : : : : +- BroadcastExchange - : : : : : : : +- CometNativeColumnarToRow - : : : : : : : +- CometProject - : : : : : : : +- CometFilter - : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : : +- CometSubqueryBroadcast + : : : : : : : +- CometBroadcastExchange + : : : : : : : +- CometProject + : : : : : : : +- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : : : +- CometBroadcastExchange : : : : : : +- CometProject : : : : : : +- CometFilter @@ -64,12 +63,11 @@ CometNativeColumnarToRow : : : : : : +- CometBroadcastHashJoin : : : : : : :- CometFilter : : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : : : : +- SubqueryBroadcast - : : : : : : : +- BroadcastExchange - : : : : : : : +- CometNativeColumnarToRow - : : : : : : : +- CometProject - : : : : : : : +- CometFilter - : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : : +- CometSubqueryBroadcast + : : : : : : : +- CometBroadcastExchange + : : : : : : : +- CometProject + : : : : : : : +- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : : : +- CometBroadcastExchange : : : : : : +- CometProject : : : : : : +- CometFilter @@ -110,12 +108,11 @@ CometNativeColumnarToRow : : : : : : +- CometBroadcastHashJoin : : : : : : :- CometFilter : : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : : : : +- SubqueryBroadcast - : : : : : : : +- BroadcastExchange - : : : : : : : +- CometNativeColumnarToRow - : : : : : : : +- CometProject - : : : : : : : +- CometFilter - : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : : +- CometSubqueryBroadcast + : : : : : : : +- CometBroadcastExchange + : : : : : : : +- CometProject + : : : : : : : +- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : : : +- CometBroadcastExchange : : : : : : +- CometProject : : : : : : +- CometFilter @@ -156,12 +153,11 @@ CometNativeColumnarToRow : : : : : : +- CometBroadcastHashJoin : : : : : : :- CometFilter : : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : : : : +- SubqueryBroadcast - : : : : : : : +- BroadcastExchange - : : : : : : : +- CometNativeColumnarToRow - : : : : : : : +- CometProject - : : : : : : : +- CometFilter - : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : : +- CometSubqueryBroadcast + : : : : : : : +- CometBroadcastExchange + : : : : : : : +- CometProject + : : : : : : : +- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : : : +- CometBroadcastExchange : : : : : : +- CometProject : : : : : : +- CometFilter @@ -202,12 +198,11 @@ CometNativeColumnarToRow : : : : : +- CometBroadcastHashJoin : : : : : :- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometNativeColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : +- CometSubqueryBroadcast + : : : : : : +- CometBroadcastExchange + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : : +- CometBroadcastExchange : : : : : +- CometProject : : : : : +- CometFilter @@ -231,4 +226,4 @@ CometNativeColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.item -Comet accelerated 216 out of 226 eligible operators (95%). Final plan contains 6 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 221 out of 226 eligible operators (97%). Final plan contains 1 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q20.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q20.native_datafusion/extended.txt index c4f54f4289..24498162ca 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q20.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q20.native_datafusion/extended.txt @@ -13,12 +13,11 @@ TakeOrderedAndProject : +- CometBroadcastHashJoin : :- CometFilter : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometSubqueryBroadcast + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim : +- CometBroadcastExchange : +- CometProject : +- CometFilter @@ -28,4 +27,4 @@ TakeOrderedAndProject +- CometFilter +- CometNativeScan parquet spark_catalog.default.date_dim -Comet accelerated 22 out of 27 eligible operators (81%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 23 out of 27 eligible operators (85%). Final plan contains 1 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q22.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q22.native_datafusion/extended.txt index a8588fd031..5d9c8333d0 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q22.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q22.native_datafusion/extended.txt @@ -13,12 +13,11 @@ TakeOrderedAndProject : : +- CometBroadcastHashJoin : : :- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.inventory - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometSubqueryBroadcast + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- CometBroadcastExchange : : +- CometProject : : +- CometFilter @@ -31,4 +30,4 @@ TakeOrderedAndProject +- CometNativeColumnarToRow +- CometNativeScan parquet spark_catalog.default.warehouse -Comet accelerated 19 out of 28 eligible operators (67%). Final plan contains 4 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 20 out of 28 eligible operators (71%). Final plan contains 3 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q22a.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q22a.native_datafusion/extended.txt index 5bd5daea84..0d8c15e00c 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q22a.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q22a.native_datafusion/extended.txt @@ -14,12 +14,11 @@ CometNativeColumnarToRow : : : +- CometBroadcastHashJoin : : : :- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.inventory - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometSubqueryBroadcast + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter @@ -45,12 +44,11 @@ CometNativeColumnarToRow : : : +- CometBroadcastHashJoin : : : :- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.inventory - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometSubqueryBroadcast + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter @@ -76,12 +74,11 @@ CometNativeColumnarToRow : : : +- CometBroadcastHashJoin : : : :- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.inventory - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometSubqueryBroadcast + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter @@ -107,12 +104,11 @@ CometNativeColumnarToRow : : : +- CometBroadcastHashJoin : : : :- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.inventory - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometSubqueryBroadcast + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter @@ -138,12 +134,11 @@ CometNativeColumnarToRow : : +- CometBroadcastHashJoin : : :- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.inventory - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometSubqueryBroadcast + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- CometBroadcastExchange : : +- CometProject : : +- CometFilter @@ -156,4 +151,4 @@ CometNativeColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.warehouse -Comet accelerated 141 out of 151 eligible operators (93%). Final plan contains 6 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 146 out of 151 eligible operators (96%). Final plan contains 1 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q27a.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q27a.native_datafusion/extended.txt index 5dc06ab86f..9c4497cbd3 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q27a.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q27a.native_datafusion/extended.txt @@ -14,12 +14,11 @@ CometNativeColumnarToRow : : : : +- CometBroadcastHashJoin : : : : :- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- CometSubqueryBroadcast + : : : : : +- CometBroadcastExchange + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : +- CometBroadcastExchange : : : : +- CometProject : : : : +- CometFilter @@ -49,12 +48,11 @@ CometNativeColumnarToRow : : : : +- CometBroadcastHashJoin : : : : :- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- CometSubqueryBroadcast + : : : : : +- CometBroadcastExchange + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : +- CometBroadcastExchange : : : : +- CometProject : : : : +- CometFilter @@ -84,12 +82,11 @@ CometNativeColumnarToRow : : : +- CometBroadcastHashJoin : : : :- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometSubqueryBroadcast + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter @@ -106,4 +103,4 @@ CometNativeColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.item -Comet accelerated 97 out of 103 eligible operators (94%). Final plan contains 4 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 100 out of 103 eligible operators (97%). Final plan contains 1 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q34.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q34.native_datafusion/extended.txt index 1b61be2937..4c0ef9abcd 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q34.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q34.native_datafusion/extended.txt @@ -15,12 +15,11 @@ CometNativeColumnarToRow : : : +- CometBroadcastHashJoin : : : :- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometSubqueryBroadcast + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter @@ -38,4 +37,4 @@ CometNativeColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.customer -Comet accelerated 35 out of 37 eligible operators (94%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 36 out of 37 eligible operators (97%). Final plan contains 1 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q35.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q35.native_datafusion/extended.txt index 2785c3d0b1..2e79f6f464 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q35.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q35.native_datafusion/extended.txt @@ -19,12 +19,11 @@ TakeOrderedAndProject : : : : +- CometProject : : : : +- CometBroadcastHashJoin : : : : :- CometNativeScan parquet spark_catalog.default.store_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- CometSubqueryBroadcast + : : : : : +- CometBroadcastExchange + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : +- CometBroadcastExchange : : : : +- CometProject : : : : +- CometFilter @@ -34,12 +33,11 @@ TakeOrderedAndProject : : : +- CometProject : : : +- CometBroadcastHashJoin : : : :- CometNativeScan parquet spark_catalog.default.web_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometSubqueryBroadcast + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter @@ -49,12 +47,11 @@ TakeOrderedAndProject : : +- CometProject : : +- CometBroadcastHashJoin : : :- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometSubqueryBroadcast + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- CometBroadcastExchange : : +- CometProject : : +- CometFilter @@ -70,4 +67,4 @@ TakeOrderedAndProject +- CometFilter +- CometNativeScan parquet spark_catalog.default.customer_demographics -Comet accelerated 41 out of 62 eligible operators (66%). Final plan contains 9 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 44 out of 62 eligible operators (70%). Final plan contains 6 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q35a.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q35a.native_datafusion/extended.txt index cba125f412..173c450c71 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q35a.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q35a.native_datafusion/extended.txt @@ -16,12 +16,11 @@ CometNativeColumnarToRow : : : +- CometProject : : : +- CometBroadcastHashJoin : : : :- CometNativeScan parquet spark_catalog.default.store_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometSubqueryBroadcast + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter @@ -31,12 +30,11 @@ CometNativeColumnarToRow : : :- CometProject : : : +- CometBroadcastHashJoin : : : :- CometNativeScan parquet spark_catalog.default.web_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometSubqueryBroadcast + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter @@ -44,12 +42,11 @@ CometNativeColumnarToRow : : +- CometProject : : +- CometBroadcastHashJoin : : :- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometSubqueryBroadcast + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- CometBroadcastExchange : : +- CometProject : : +- CometFilter @@ -63,4 +60,4 @@ CometNativeColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.customer_demographics -Comet accelerated 54 out of 60 eligible operators (90%). Final plan contains 4 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 57 out of 60 eligible operators (95%). Final plan contains 1 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q36a.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q36a.native_datafusion/extended.txt index 0a69bca968..be0d09e04e 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q36a.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q36a.native_datafusion/extended.txt @@ -19,12 +19,11 @@ TakeOrderedAndProject : : : +- CometBroadcastHashJoin : : : :- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometSubqueryBroadcast + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter @@ -51,12 +50,11 @@ TakeOrderedAndProject : : : +- CometBroadcastHashJoin : : : :- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometSubqueryBroadcast + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter @@ -83,12 +81,11 @@ TakeOrderedAndProject : : +- CometBroadcastHashJoin : : :- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometSubqueryBroadcast + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- CometBroadcastExchange : : +- CometProject : : +- CometFilter @@ -102,4 +99,4 @@ TakeOrderedAndProject +- CometFilter +- CometNativeScan parquet spark_catalog.default.store -Comet accelerated 90 out of 99 eligible operators (90%). Final plan contains 4 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 93 out of 99 eligible operators (93%). Final plan contains 1 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q47.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q47.native_datafusion/extended.txt index 0886059452..54fa1daf2e 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q47.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q47.native_datafusion/extended.txt @@ -26,11 +26,10 @@ TakeOrderedAndProject : : : : +- CometBroadcastExchange : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometSubqueryBroadcast + : : : : +- CometBroadcastExchange + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- CometBroadcastExchange : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -58,11 +57,10 @@ TakeOrderedAndProject : : : +- CometBroadcastExchange : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometSubqueryBroadcast + : : : +- CometBroadcastExchange + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- CometBroadcastExchange : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -90,11 +88,10 @@ TakeOrderedAndProject : : +- CometBroadcastExchange : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometSubqueryBroadcast + : : +- CometBroadcastExchange + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim : +- CometBroadcastExchange : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -102,4 +99,4 @@ TakeOrderedAndProject +- CometFilter +- CometNativeScan parquet spark_catalog.default.store -Comet accelerated 75 out of 97 eligible operators (77%). Final plan contains 6 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 78 out of 97 eligible operators (80%). Final plan contains 3 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q49.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q49.native_datafusion/extended.txt index 6d7dadcde9..f2ae44fb86 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q49.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q49.native_datafusion/extended.txt @@ -23,12 +23,11 @@ CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometSubqueryBroadcast + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- CometProject : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.web_returns @@ -55,12 +54,11 @@ CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometSubqueryBroadcast + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- CometProject : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.catalog_returns @@ -87,12 +85,11 @@ CometNativeColumnarToRow : : +- CometProject : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometSubqueryBroadcast + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim : +- CometProject : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.store_returns @@ -101,4 +98,4 @@ CometNativeColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.date_dim -Comet accelerated 72 out of 95 eligible operators (75%). Final plan contains 7 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 75 out of 95 eligible operators (78%). Final plan contains 4 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q51a.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q51a.native_datafusion/extended.txt index d9b9186c1e..a3400c5ecf 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q51a.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q51a.native_datafusion/extended.txt @@ -31,12 +31,11 @@ TakeOrderedAndProject : : : +- CometBroadcastHashJoin : : : :- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometSubqueryBroadcast + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter @@ -54,12 +53,11 @@ TakeOrderedAndProject : : +- CometBroadcastHashJoin : : :- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometSubqueryBroadcast + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- CometBroadcastExchange : : +- CometProject : : +- CometFilter @@ -84,12 +82,11 @@ TakeOrderedAndProject : : +- CometBroadcastHashJoin : : :- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometSubqueryBroadcast + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- CometBroadcastExchange : : +- CometProject : : +- CometFilter @@ -107,12 +104,11 @@ TakeOrderedAndProject : +- CometBroadcastHashJoin : :- CometFilter : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometSubqueryBroadcast + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim : +- CometBroadcastExchange : +- CometProject : +- CometFilter @@ -146,12 +142,11 @@ TakeOrderedAndProject : : +- CometBroadcastHashJoin : : :- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometSubqueryBroadcast + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- CometBroadcastExchange : : +- CometProject : : +- CometFilter @@ -169,12 +164,11 @@ TakeOrderedAndProject : +- CometBroadcastHashJoin : :- CometFilter : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometSubqueryBroadcast + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim : +- CometBroadcastExchange : +- CometProject : +- CometFilter @@ -199,12 +193,11 @@ TakeOrderedAndProject : +- CometBroadcastHashJoin : :- CometFilter : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometSubqueryBroadcast + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim : +- CometBroadcastExchange : +- CometProject : +- CometFilter @@ -222,15 +215,14 @@ TakeOrderedAndProject +- CometBroadcastHashJoin :- CometFilter : +- CometNativeScan parquet spark_catalog.default.store_sales - : +- SubqueryBroadcast - : +- BroadcastExchange - : +- CometNativeColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometSubqueryBroadcast + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.date_dim +- CometBroadcastExchange +- CometProject +- CometFilter +- CometNativeScan parquet spark_catalog.default.date_dim -Comet accelerated 150 out of 212 eligible operators (70%). Final plan contains 22 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 158 out of 212 eligible operators (74%). Final plan contains 14 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q57.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q57.native_datafusion/extended.txt index 024bfeeeea..aa4ccf3246 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q57.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q57.native_datafusion/extended.txt @@ -26,11 +26,10 @@ TakeOrderedAndProject : : : : +- CometBroadcastExchange : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometSubqueryBroadcast + : : : : +- CometBroadcastExchange + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- CometBroadcastExchange : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -58,11 +57,10 @@ TakeOrderedAndProject : : : +- CometBroadcastExchange : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometSubqueryBroadcast + : : : +- CometBroadcastExchange + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- CometBroadcastExchange : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -90,11 +88,10 @@ TakeOrderedAndProject : : +- CometBroadcastExchange : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometSubqueryBroadcast + : : +- CometBroadcastExchange + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim : +- CometBroadcastExchange : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -102,4 +99,4 @@ TakeOrderedAndProject +- CometFilter +- CometNativeScan parquet spark_catalog.default.call_center -Comet accelerated 75 out of 97 eligible operators (77%). Final plan contains 6 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 78 out of 97 eligible operators (80%). Final plan contains 3 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q5a.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q5a.native_datafusion/extended.txt index 9cc06fb361..b472530c10 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q5a.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q5a.native_datafusion/extended.txt @@ -19,21 +19,19 @@ CometNativeColumnarToRow : : : : :- CometProject : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- CometSubqueryBroadcast + : : : : : +- CometBroadcastExchange + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : +- CometProject : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.store_returns - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometSubqueryBroadcast + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter @@ -53,21 +51,19 @@ CometNativeColumnarToRow : : : : :- CometProject : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- CometSubqueryBroadcast + : : : : : +- CometBroadcastExchange + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : +- CometProject : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.catalog_returns - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometSubqueryBroadcast + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter @@ -87,22 +83,20 @@ CometNativeColumnarToRow : : : :- CometProject : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometSubqueryBroadcast + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- CometProject : : : +- CometBroadcastHashJoin : : : :- CometBroadcastExchange : : : : +- CometNativeScan parquet spark_catalog.default.web_returns - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometSubqueryBroadcast + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.web_sales @@ -132,21 +126,19 @@ CometNativeColumnarToRow : : : : :- CometProject : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- CometSubqueryBroadcast + : : : : : +- CometBroadcastExchange + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : +- CometProject : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.store_returns - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometSubqueryBroadcast + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter @@ -166,21 +158,19 @@ CometNativeColumnarToRow : : : : :- CometProject : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- CometSubqueryBroadcast + : : : : : +- CometBroadcastExchange + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : +- CometProject : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.catalog_returns - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometSubqueryBroadcast + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter @@ -200,22 +190,20 @@ CometNativeColumnarToRow : : : :- CometProject : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometSubqueryBroadcast + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- CometProject : : : +- CometBroadcastHashJoin : : : :- CometBroadcastExchange : : : : +- CometNativeScan parquet spark_catalog.default.web_returns - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometSubqueryBroadcast + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.web_sales @@ -245,21 +233,19 @@ CometNativeColumnarToRow : : : :- CometProject : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometSubqueryBroadcast + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.store_returns - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometSubqueryBroadcast + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- CometBroadcastExchange : : +- CometProject : : +- CometFilter @@ -279,21 +265,19 @@ CometNativeColumnarToRow : : : :- CometProject : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometSubqueryBroadcast + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.catalog_returns - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometSubqueryBroadcast + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- CometBroadcastExchange : : +- CometProject : : +- CometFilter @@ -313,22 +297,20 @@ CometNativeColumnarToRow : : :- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometSubqueryBroadcast + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- CometProject : : +- CometBroadcastHashJoin : : :- CometBroadcastExchange : : : +- CometNativeScan parquet spark_catalog.default.web_returns - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometSubqueryBroadcast + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- CometProject : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.web_sales @@ -341,4 +323,4 @@ CometNativeColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.web_site -Comet accelerated 287 out of 323 eligible operators (88%). Final plan contains 19 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 305 out of 323 eligible operators (94%). Final plan contains 1 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q6.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q6.native_datafusion/extended.txt index 1cb21cb650..4b31bf751a 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q6.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q6.native_datafusion/extended.txt @@ -21,24 +21,30 @@ CometNativeColumnarToRow : : +- CometBroadcastExchange : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometProject - : : +- CometFilter - : : : +- Subquery - : : : +- CometNativeColumnarToRow - : : : +- CometHashAggregate - : : : +- CometExchange - : : : +- CometHashAggregate - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometSubqueryBroadcast + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : : +- Subquery + : : : +- CometNativeColumnarToRow + : : : +- CometHashAggregate + : : : +- CometExchange + : : : +- CometHashAggregate + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometNativeScan parquet spark_catalog.default.date_dim : +- CometBroadcastExchange : +- CometProject : +- CometFilter - : : +- ReusedSubquery + : : +- Subquery + : : +- CometNativeColumnarToRow + : : +- CometHashAggregate + : : +- CometExchange + : : +- CometHashAggregate + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim : +- CometNativeScan parquet spark_catalog.default.date_dim +- CometBroadcastExchange +- CometProject @@ -54,4 +60,4 @@ CometNativeColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.item -Comet accelerated 48 out of 52 eligible operators (92%). Final plan contains 3 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 55 out of 58 eligible operators (94%). Final plan contains 3 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q64.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q64.native_datafusion/extended.txt index f1fe6fb3e3..14ffe4eab9 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q64.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q64.native_datafusion/extended.txt @@ -46,11 +46,10 @@ CometNativeColumnarToRow : : : : : : : : : : : : : : : : : :- CometBroadcastExchange : : : : : : : : : : : : : : : : : : +- CometFilter : : : : : : : : : : : : : : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : : : : : : : : : : : : : : : : +- SubqueryBroadcast - : : : : : : : : : : : : : : : : : : +- BroadcastExchange - : : : : : : : : : : : : : : : : : : +- CometNativeColumnarToRow - : : : : : : : : : : : : : : : : : : +- CometFilter - : : : : : : : : : : : : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : : : : : : : : : : : : : +- CometSubqueryBroadcast + : : : : : : : : : : : : : : : : : : +- CometBroadcastExchange + : : : : : : : : : : : : : : : : : : +- CometFilter + : : : : : : : : : : : : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : : : : : : : : : : : : : : +- CometProject : : : : : : : : : : : : : : : : : +- CometFilter : : : : : : : : : : : : : : : : : +- CometNativeScan parquet spark_catalog.default.store_returns @@ -166,11 +165,10 @@ CometNativeColumnarToRow : : : : : : : : : : : : : : : : :- CometBroadcastExchange : : : : : : : : : : : : : : : : : +- CometFilter : : : : : : : : : : : : : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : : : : : : : : : : : : : : : +- SubqueryBroadcast - : : : : : : : : : : : : : : : : : +- BroadcastExchange - : : : : : : : : : : : : : : : : : +- CometNativeColumnarToRow - : : : : : : : : : : : : : : : : : +- CometFilter - : : : : : : : : : : : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : : : : : : : : : : : : +- CometSubqueryBroadcast + : : : : : : : : : : : : : : : : : +- CometBroadcastExchange + : : : : : : : : : : : : : : : : : +- CometFilter + : : : : : : : : : : : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : : : : : : : : : : : : : +- CometProject : : : : : : : : : : : : : : : : +- CometFilter : : : : : : : : : : : : : : : : +- CometNativeScan parquet spark_catalog.default.store_returns @@ -244,4 +242,4 @@ CometNativeColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.item -Comet accelerated 238 out of 242 eligible operators (98%). Final plan contains 3 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 240 out of 242 eligible operators (99%). Final plan contains 1 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q67a.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q67a.native_datafusion/extended.txt index cdcf018198..19b0999bb5 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q67a.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q67a.native_datafusion/extended.txt @@ -16,12 +16,11 @@ TakeOrderedAndProject : : : +- CometBroadcastHashJoin : : : :- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometSubqueryBroadcast + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter @@ -48,12 +47,11 @@ TakeOrderedAndProject : : : +- CometBroadcastHashJoin : : : :- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometSubqueryBroadcast + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter @@ -80,12 +78,11 @@ TakeOrderedAndProject : : : +- CometBroadcastHashJoin : : : :- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometSubqueryBroadcast + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter @@ -112,12 +109,11 @@ TakeOrderedAndProject : : : +- CometBroadcastHashJoin : : : :- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometSubqueryBroadcast + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter @@ -144,12 +140,11 @@ TakeOrderedAndProject : : : +- CometBroadcastHashJoin : : : :- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometSubqueryBroadcast + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter @@ -176,12 +171,11 @@ TakeOrderedAndProject : : : +- CometBroadcastHashJoin : : : :- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometSubqueryBroadcast + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter @@ -208,12 +202,11 @@ TakeOrderedAndProject : : : +- CometBroadcastHashJoin : : : :- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometSubqueryBroadcast + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter @@ -240,12 +233,11 @@ TakeOrderedAndProject : : : +- CometBroadcastHashJoin : : : :- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometSubqueryBroadcast + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter @@ -272,12 +264,11 @@ TakeOrderedAndProject : : +- CometBroadcastHashJoin : : :- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometSubqueryBroadcast + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- CometBroadcastExchange : : +- CometProject : : +- CometFilter @@ -291,4 +282,4 @@ TakeOrderedAndProject +- CometFilter +- CometNativeScan parquet spark_catalog.default.item -Comet accelerated 261 out of 282 eligible operators (92%). Final plan contains 10 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 270 out of 282 eligible operators (95%). Final plan contains 1 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q70a.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q70a.native_datafusion/extended.txt index 9f507b48bf..5e1387f550 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q70a.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q70a.native_datafusion/extended.txt @@ -19,12 +19,11 @@ TakeOrderedAndProject : : +- CometBroadcastHashJoin : : :- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometSubqueryBroadcast + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- CometBroadcastExchange : : +- CometProject : : +- CometFilter @@ -50,12 +49,11 @@ TakeOrderedAndProject : : +- CometBroadcastHashJoin : : :- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometSubqueryBroadcast + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- CometBroadcastExchange : : +- CometProject : : +- CometFilter @@ -79,12 +77,11 @@ TakeOrderedAndProject : : +- CometBroadcastHashJoin : : :- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometSubqueryBroadcast + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- CometBroadcastExchange : : +- CometProject : : +- CometFilter @@ -110,12 +107,11 @@ TakeOrderedAndProject : : +- CometBroadcastHashJoin : : :- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometSubqueryBroadcast + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- CometBroadcastExchange : : +- CometProject : : +- CometFilter @@ -139,12 +135,11 @@ TakeOrderedAndProject : +- CometBroadcastHashJoin : :- CometFilter : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometSubqueryBroadcast + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim : +- CometBroadcastExchange : +- CometProject : +- CometFilter @@ -170,12 +165,11 @@ TakeOrderedAndProject : +- CometBroadcastHashJoin : :- CometFilter : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometSubqueryBroadcast + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim : +- CometBroadcastExchange : +- CometProject : +- CometFilter @@ -185,4 +179,4 @@ TakeOrderedAndProject +- CometFilter +- CometNativeScan parquet spark_catalog.default.date_dim -Comet accelerated 111 out of 165 eligible operators (67%). Final plan contains 21 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 117 out of 165 eligible operators (70%). Final plan contains 15 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q72.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q72.native_datafusion/extended.txt index 623a68ddab..d443b31d1e 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q72.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q72.native_datafusion/extended.txt @@ -27,12 +27,11 @@ CometNativeColumnarToRow : : : : : : : : : +- CometBroadcastHashJoin : : : : : : : : : :- CometFilter : : : : : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : : : : : : : +- SubqueryBroadcast - : : : : : : : : : : +- BroadcastExchange - : : : : : : : : : : +- CometNativeColumnarToRow - : : : : : : : : : : +- CometProject - : : : : : : : : : : +- CometFilter - : : : : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : : : : : +- CometSubqueryBroadcast + : : : : : : : : : : +- CometBroadcastExchange + : : : : : : : : : : +- CometProject + : : : : : : : : : : +- CometFilter + : : : : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : : : : : : +- CometBroadcastExchange : : : : : : : : : +- CometFilter : : : : : : : : : +- CometNativeScan parquet spark_catalog.default.inventory @@ -69,4 +68,4 @@ CometNativeColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.catalog_returns -Comet accelerated 66 out of 68 eligible operators (97%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 67 out of 68 eligible operators (98%). Final plan contains 1 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q74.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q74.native_datafusion/extended.txt index 51a3f5ca94..7768138645 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q74.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q74.native_datafusion/extended.txt @@ -19,11 +19,10 @@ CometNativeColumnarToRow : : : : +- CometBroadcastExchange : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometSubqueryBroadcast + : : : : +- CometBroadcastExchange + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- CometBroadcastExchange : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -41,11 +40,10 @@ CometNativeColumnarToRow : : : +- CometBroadcastExchange : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometSubqueryBroadcast + : : : +- CometBroadcastExchange + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- CometBroadcastExchange : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -64,11 +62,10 @@ CometNativeColumnarToRow : : +- CometBroadcastExchange : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometSubqueryBroadcast + : : +- CometBroadcastExchange + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim : +- CometBroadcastExchange : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -86,13 +83,12 @@ CometNativeColumnarToRow : +- CometBroadcastExchange : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.web_sales - : +- SubqueryBroadcast - : +- BroadcastExchange - : +- CometNativeColumnarToRow - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometSubqueryBroadcast + : +- CometBroadcastExchange + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.date_dim +- CometBroadcastExchange +- CometFilter +- CometNativeScan parquet spark_catalog.default.date_dim -Comet accelerated 83 out of 91 eligible operators (91%). Final plan contains 5 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 87 out of 91 eligible operators (95%). Final plan contains 1 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q75.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q75.native_datafusion/extended.txt index a943308f1e..ee7c0af216 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q75.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q75.native_datafusion/extended.txt @@ -22,11 +22,10 @@ CometNativeColumnarToRow : : : : +- CometBroadcastHashJoin : : : : :- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- CometSubqueryBroadcast + : : : : : +- CometBroadcastExchange + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : +- CometBroadcastExchange : : : : +- CometProject : : : : +- CometFilter @@ -49,11 +48,10 @@ CometNativeColumnarToRow : : : : +- CometBroadcastHashJoin : : : : :- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- CometSubqueryBroadcast + : : : : : +- CometBroadcastExchange + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : +- CometBroadcastExchange : : : : +- CometProject : : : : +- CometFilter @@ -76,11 +74,10 @@ CometNativeColumnarToRow : : : +- CometBroadcastHashJoin : : : :- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometSubqueryBroadcast + : : : : +- CometBroadcastExchange + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter @@ -113,11 +110,10 @@ CometNativeColumnarToRow : : : +- CometBroadcastHashJoin : : : :- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometSubqueryBroadcast + : : : : +- CometBroadcastExchange + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter @@ -140,11 +136,10 @@ CometNativeColumnarToRow : : : +- CometBroadcastHashJoin : : : :- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometSubqueryBroadcast + : : : : +- CometBroadcastExchange + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter @@ -167,11 +162,10 @@ CometNativeColumnarToRow : : +- CometBroadcastHashJoin : : :- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometSubqueryBroadcast + : : : +- CometBroadcastExchange + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- CometBroadcastExchange : : +- CometProject : : +- CometFilter @@ -185,4 +179,4 @@ CometNativeColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.web_returns -Comet accelerated 167 out of 179 eligible operators (93%). Final plan contains 7 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 173 out of 179 eligible operators (96%). Final plan contains 1 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q77a.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q77a.native_datafusion/extended.txt index 24407fc0a2..6621cfd8f8 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q77a.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q77a.native_datafusion/extended.txt @@ -21,12 +21,11 @@ CometNativeColumnarToRow : : : : +- CometBroadcastHashJoin : : : : :- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- CometSubqueryBroadcast + : : : : : +- CometBroadcastExchange + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : +- CometBroadcastExchange : : : : +- CometProject : : : : +- CometFilter @@ -44,12 +43,11 @@ CometNativeColumnarToRow : : : +- CometBroadcastHashJoin : : : :- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.store_returns - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometSubqueryBroadcast + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter @@ -67,12 +65,11 @@ CometNativeColumnarToRow : : : +- CometProject : : : +- CometBroadcastHashJoin : : : :- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometSubqueryBroadcast + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter @@ -84,12 +81,11 @@ CometNativeColumnarToRow : : +- CometProject : : +- CometBroadcastHashJoin : : :- CometNativeScan parquet spark_catalog.default.catalog_returns - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometSubqueryBroadcast + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- CometBroadcastExchange : : +- CometProject : : +- CometFilter @@ -106,12 +102,11 @@ CometNativeColumnarToRow : : : +- CometBroadcastHashJoin : : : :- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometSubqueryBroadcast + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter @@ -129,12 +124,11 @@ CometNativeColumnarToRow : : +- CometBroadcastHashJoin : : :- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.web_returns - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometSubqueryBroadcast + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- CometBroadcastExchange : : +- CometProject : : +- CometFilter @@ -163,12 +157,11 @@ CometNativeColumnarToRow : : : : +- CometBroadcastHashJoin : : : : :- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- CometSubqueryBroadcast + : : : : : +- CometBroadcastExchange + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : +- CometBroadcastExchange : : : : +- CometProject : : : : +- CometFilter @@ -186,12 +179,11 @@ CometNativeColumnarToRow : : : +- CometBroadcastHashJoin : : : :- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.store_returns - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometSubqueryBroadcast + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter @@ -209,12 +201,11 @@ CometNativeColumnarToRow : : : +- CometProject : : : +- CometBroadcastHashJoin : : : :- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometSubqueryBroadcast + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter @@ -226,12 +217,11 @@ CometNativeColumnarToRow : : +- CometProject : : +- CometBroadcastHashJoin : : :- CometNativeScan parquet spark_catalog.default.catalog_returns - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometSubqueryBroadcast + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- CometBroadcastExchange : : +- CometProject : : +- CometFilter @@ -248,12 +238,11 @@ CometNativeColumnarToRow : : : +- CometBroadcastHashJoin : : : :- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometSubqueryBroadcast + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter @@ -271,12 +260,11 @@ CometNativeColumnarToRow : : +- CometBroadcastHashJoin : : :- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.web_returns - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometSubqueryBroadcast + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- CometBroadcastExchange : : +- CometProject : : +- CometFilter @@ -305,12 +293,11 @@ CometNativeColumnarToRow : : : +- CometBroadcastHashJoin : : : :- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometSubqueryBroadcast + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter @@ -328,12 +315,11 @@ CometNativeColumnarToRow : : +- CometBroadcastHashJoin : : :- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.store_returns - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometSubqueryBroadcast + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- CometBroadcastExchange : : +- CometProject : : +- CometFilter @@ -351,12 +337,11 @@ CometNativeColumnarToRow : : +- CometProject : : +- CometBroadcastHashJoin : : :- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometSubqueryBroadcast + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- CometBroadcastExchange : : +- CometProject : : +- CometFilter @@ -368,12 +353,11 @@ CometNativeColumnarToRow : +- CometProject : +- CometBroadcastHashJoin : :- CometNativeScan parquet spark_catalog.default.catalog_returns - : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometSubqueryBroadcast + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim : +- CometBroadcastExchange : +- CometProject : +- CometFilter @@ -390,12 +374,11 @@ CometNativeColumnarToRow : : +- CometBroadcastHashJoin : : :- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometSubqueryBroadcast + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- CometBroadcastExchange : : +- CometProject : : +- CometFilter @@ -413,12 +396,11 @@ CometNativeColumnarToRow : +- CometBroadcastHashJoin : :- CometFilter : : +- CometNativeScan parquet spark_catalog.default.web_returns - : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometSubqueryBroadcast + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim : +- CometBroadcastExchange : +- CometProject : +- CometFilter @@ -427,4 +409,4 @@ CometNativeColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.web_page -Comet accelerated 332 out of 392 eligible operators (84%). Final plan contains 36 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 350 out of 392 eligible operators (89%). Final plan contains 18 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q78.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q78.native_datafusion/extended.txt index ebfdcc3c62..8426b9653c 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q78.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q78.native_datafusion/extended.txt @@ -17,11 +17,10 @@ TakeOrderedAndProject : : : : +- CometExchange : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometSubqueryBroadcast + : : : : +- CometBroadcastExchange + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- CometSort : : : +- CometExchange : : : +- CometProject @@ -44,11 +43,10 @@ TakeOrderedAndProject : : : +- CometExchange : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometSubqueryBroadcast + : : : +- CometBroadcastExchange + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- CometSort : : +- CometExchange : : +- CometProject @@ -71,11 +69,10 @@ TakeOrderedAndProject : : +- CometExchange : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometSubqueryBroadcast + : : +- CometBroadcastExchange + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim : +- CometSort : +- CometExchange : +- CometProject @@ -85,4 +82,4 @@ TakeOrderedAndProject +- CometFilter +- CometNativeScan parquet spark_catalog.default.date_dim -Comet accelerated 74 out of 82 eligible operators (90%). Final plan contains 4 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 77 out of 82 eligible operators (93%). Final plan contains 1 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q80a.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q80a.native_datafusion/extended.txt index ec566b42de..dced05333a 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q80a.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q80a.native_datafusion/extended.txt @@ -25,12 +25,11 @@ CometNativeColumnarToRow : : : : : : : +- CometExchange : : : : : : : +- CometFilter : : : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : : : : : +- SubqueryBroadcast - : : : : : : : +- BroadcastExchange - : : : : : : : +- CometNativeColumnarToRow - : : : : : : : +- CometProject - : : : : : : : +- CometFilter - : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : : +- CometSubqueryBroadcast + : : : : : : : +- CometBroadcastExchange + : : : : : : : +- CometProject + : : : : : : : +- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : : : +- CometSort : : : : : : +- CometExchange : : : : : : +- CometProject @@ -69,12 +68,11 @@ CometNativeColumnarToRow : : : : : : : +- CometExchange : : : : : : : +- CometFilter : : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : : : : +- SubqueryBroadcast - : : : : : : : +- BroadcastExchange - : : : : : : : +- CometNativeColumnarToRow - : : : : : : : +- CometProject - : : : : : : : +- CometFilter - : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : : +- CometSubqueryBroadcast + : : : : : : : +- CometBroadcastExchange + : : : : : : : +- CometProject + : : : : : : : +- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : : : +- CometSort : : : : : : +- CometExchange : : : : : : +- CometProject @@ -113,12 +111,11 @@ CometNativeColumnarToRow : : : : : : +- CometExchange : : : : : : +- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometNativeColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : +- CometSubqueryBroadcast + : : : : : : +- CometBroadcastExchange + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : : +- CometSort : : : : : +- CometExchange : : : : : +- CometProject @@ -164,12 +161,11 @@ CometNativeColumnarToRow : : : : : : : +- CometExchange : : : : : : : +- CometFilter : : : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : : : : : +- SubqueryBroadcast - : : : : : : : +- BroadcastExchange - : : : : : : : +- CometNativeColumnarToRow - : : : : : : : +- CometProject - : : : : : : : +- CometFilter - : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : : +- CometSubqueryBroadcast + : : : : : : : +- CometBroadcastExchange + : : : : : : : +- CometProject + : : : : : : : +- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : : : +- CometSort : : : : : : +- CometExchange : : : : : : +- CometProject @@ -208,12 +204,11 @@ CometNativeColumnarToRow : : : : : : : +- CometExchange : : : : : : : +- CometFilter : : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : : : : +- SubqueryBroadcast - : : : : : : : +- BroadcastExchange - : : : : : : : +- CometNativeColumnarToRow - : : : : : : : +- CometProject - : : : : : : : +- CometFilter - : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : : +- CometSubqueryBroadcast + : : : : : : : +- CometBroadcastExchange + : : : : : : : +- CometProject + : : : : : : : +- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : : : +- CometSort : : : : : : +- CometExchange : : : : : : +- CometProject @@ -252,12 +247,11 @@ CometNativeColumnarToRow : : : : : : +- CometExchange : : : : : : +- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometNativeColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : +- CometSubqueryBroadcast + : : : : : : +- CometBroadcastExchange + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : : +- CometSort : : : : : +- CometExchange : : : : : +- CometProject @@ -303,12 +297,11 @@ CometNativeColumnarToRow : : : : : : +- CometExchange : : : : : : +- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometNativeColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : +- CometSubqueryBroadcast + : : : : : : +- CometBroadcastExchange + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : : +- CometSort : : : : : +- CometExchange : : : : : +- CometProject @@ -347,12 +340,11 @@ CometNativeColumnarToRow : : : : : : +- CometExchange : : : : : : +- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometNativeColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : +- CometSubqueryBroadcast + : : : : : : +- CometBroadcastExchange + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : : +- CometSort : : : : : +- CometExchange : : : : : +- CometProject @@ -391,12 +383,11 @@ CometNativeColumnarToRow : : : : : +- CometExchange : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- CometSubqueryBroadcast + : : : : : +- CometBroadcastExchange + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : +- CometSort : : : : +- CometExchange : : : : +- CometProject @@ -419,4 +410,4 @@ CometNativeColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.promotion -Comet accelerated 392 out of 410 eligible operators (95%). Final plan contains 10 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 401 out of 410 eligible operators (97%). Final plan contains 1 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q86a.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q86a.native_datafusion/extended.txt index c1df720935..4c66209a5e 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q86a.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q86a.native_datafusion/extended.txt @@ -17,12 +17,11 @@ TakeOrderedAndProject : : +- CometBroadcastHashJoin : : :- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometSubqueryBroadcast + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- CometBroadcastExchange : : +- CometProject : : +- CometFilter @@ -43,12 +42,11 @@ TakeOrderedAndProject : : +- CometBroadcastHashJoin : : :- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometSubqueryBroadcast + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- CometBroadcastExchange : : +- CometProject : : +- CometFilter @@ -69,12 +67,11 @@ TakeOrderedAndProject : +- CometBroadcastHashJoin : :- CometFilter : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometSubqueryBroadcast + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim : +- CometBroadcastExchange : +- CometProject : +- CometFilter @@ -84,4 +81,4 @@ TakeOrderedAndProject +- CometFilter +- CometNativeScan parquet spark_catalog.default.item -Comet accelerated 72 out of 81 eligible operators (88%). Final plan contains 4 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 75 out of 81 eligible operators (92%). Final plan contains 1 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q98.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q98.native_datafusion/extended.txt index 6c2b043008..ccf9f25336 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q98.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q98.native_datafusion/extended.txt @@ -15,12 +15,11 @@ CometNativeColumnarToRow : +- CometBroadcastHashJoin : :- CometFilter : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometSubqueryBroadcast + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim : +- CometBroadcastExchange : +- CometProject : +- CometFilter @@ -30,4 +29,4 @@ CometNativeColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.date_dim -Comet accelerated 24 out of 28 eligible operators (85%). Final plan contains 3 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 25 out of 28 eligible operators (89%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file From bcd2eace56620aa9039a050b22b64857a2887ee6 Mon Sep 17 00:00:00 2001 From: Matt Butrovich Date: Tue, 21 Apr 2026 10:03:44 -0400 Subject: [PATCH 12/21] Fix format issues. --- .../src/main/scala/org/apache/comet/rules/CometExecRule.scala | 4 ++-- .../org/apache/spark/sql/comet/CometNativeScanExec.scala | 4 ++-- .../apache/spark/sql/comet/CometSubqueryBroadcastExec.scala | 4 ++-- 3 files changed, 6 insertions(+), 6 deletions(-) diff --git a/spark/src/main/scala/org/apache/comet/rules/CometExecRule.scala b/spark/src/main/scala/org/apache/comet/rules/CometExecRule.scala index 0da76475bc..1a1361dd27 100644 --- a/spark/src/main/scala/org/apache/comet/rules/CometExecRule.scala +++ b/spark/src/main/scala/org/apache/comet/rules/CometExecRule.scala @@ -339,8 +339,8 @@ case class CometExecRule(session: SparkSession) } if (cometChild.isInstanceOf[CometNativeExec]) { logInfo( - s"Converting SubqueryBroadcastExec to " + - s"CometSubqueryBroadcastExec for DPP exchange reuse") + "Converting SubqueryBroadcastExec to " + + "CometSubqueryBroadcastExec for DPP exchange reuse") val cometBroadcast = CometBroadcastExchangeExec(b, b.output, b.mode, cometChild) val cometSub = CometSubqueryBroadcastExec( sub.name, diff --git a/spark/src/main/scala/org/apache/spark/sql/comet/CometNativeScanExec.scala b/spark/src/main/scala/org/apache/spark/sql/comet/CometNativeScanExec.scala index aaa3e8c104..cff9267cb4 100644 --- a/spark/src/main/scala/org/apache/spark/sql/comet/CometNativeScanExec.scala +++ b/spark/src/main/scala/org/apache/spark/sql/comet/CometNativeScanExec.scala @@ -185,7 +185,7 @@ case class CometNativeScanExec( logDebug(s"Resolving DPP subquery: plan=${e.plan.getClass.getSimpleName}") try { e.updateResult() - logDebug(s"DPP subquery resolved successfully") + logDebug("DPP subquery resolved successfully") } catch { case ex: Exception => logError(s"DPP subquery resolution failed: ${ex.getMessage}") @@ -200,7 +200,7 @@ case class CometNativeScanExec( scan.partitionFilters.foreach { case DynamicPruningExpression(e: InSubqueryExec) if e.values().isEmpty => logDebug( - s"Resolving CometScanExec DPP subquery: " + + "Resolving CometScanExec DPP subquery: " + s"plan=${e.plan.getClass.getSimpleName}") e.updateResult() case _ => diff --git a/spark/src/main/scala/org/apache/spark/sql/comet/CometSubqueryBroadcastExec.scala b/spark/src/main/scala/org/apache/spark/sql/comet/CometSubqueryBroadcastExec.scala index b184eea7d6..226854d18c 100644 --- a/spark/src/main/scala/org/apache/spark/sql/comet/CometSubqueryBroadcastExec.scala +++ b/spark/src/main/scala/org/apache/spark/sql/comet/CometSubqueryBroadcastExec.scala @@ -31,7 +31,7 @@ import org.apache.spark.sql.catalyst.plans.QueryPlan import org.apache.spark.sql.comet.util.Utils import org.apache.spark.sql.errors.QueryExecutionErrors import org.apache.spark.sql.execution._ -import org.apache.spark.sql.execution.metric.SQLMetrics +import org.apache.spark.sql.execution.metric.{SQLMetric, SQLMetrics} import org.apache.spark.sql.internal.{SQLConf, StaticSQLConf} import org.apache.spark.util.ThreadUtils import org.apache.spark.util.io.ChunkedByteBuffer @@ -72,7 +72,7 @@ case class CometSubqueryBroadcastExec( } } - override lazy val metrics = Map( + override lazy val metrics: Map[String, SQLMetric] = Map( "numOutputRows" -> SQLMetrics.createMetric(sparkContext, "number of output rows"), "dataSize" -> SQLMetrics.createMetric(sparkContext, "data size (bytes)"), "collectTime" -> SQLMetrics.createMetric(sparkContext, "time to collect (ms)")) From b8ff4f5783aac57aa89bd162d8f2fd60f4047d4e Mon Sep 17 00:00:00 2001 From: Matt Butrovich Date: Tue, 21 Apr 2026 10:37:44 -0400 Subject: [PATCH 13/21] Fix unnecessary includes in 3.4.3 and 4.0.1 diffs. --- dev/diffs/3.4.3.diff | 7 +++---- dev/diffs/4.0.1.diff | 7 +++---- 2 files changed, 6 insertions(+), 8 deletions(-) diff --git a/dev/diffs/3.4.3.diff b/dev/diffs/3.4.3.diff index ffa6f6cbae..ae9490ddb3 100644 --- a/dev/diffs/3.4.3.diff +++ b/dev/diffs/3.4.3.diff @@ -1,5 +1,5 @@ diff --git a/pom.xml b/pom.xml -index d3544881af1..377683b10c5 100644 +index d3544881af1..d075572c5b3 100644 --- a/pom.xml +++ b/pom.xml @@ -148,6 +148,8 @@ @@ -1305,17 +1305,16 @@ index 4b3d3a4b805..56e1e0e6f16 100644 setupTestData() diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/RemoveRedundantProjectsSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/RemoveRedundantProjectsSuite.scala -index 9e9d717db3b..ad41c8f3901 100644 +index 9e9d717db3b..bb04190c3ca 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/RemoveRedundantProjectsSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/RemoveRedundantProjectsSuite.scala @@ -17,7 +17,10 @@ package org.apache.spark.sql.execution --import org.apache.spark.sql.{DataFrame, QueryTest, Row} +import org.apache.comet.CometConf + -+import org.apache.spark.sql.{DataFrame, IgnoreComet, QueryTest, Row} + import org.apache.spark.sql.{DataFrame, QueryTest, Row} +import org.apache.spark.sql.comet.CometProjectExec import org.apache.spark.sql.connector.SimpleWritableDataSource import org.apache.spark.sql.execution.adaptive.{AdaptiveSparkPlanHelper, DisableAdaptiveExecutionSuite, EnableAdaptiveExecutionSuite} diff --git a/dev/diffs/4.0.1.diff b/dev/diffs/4.0.1.diff index c684d9d35c..37b72dc05b 100644 --- a/dev/diffs/4.0.1.diff +++ b/dev/diffs/4.0.1.diff @@ -39,7 +39,7 @@ index 6c51bd4ff2e..e72ec1d26e2 100644 withSpark(sc) { sc => TestUtils.waitUntilExecutorsUp(sc, 2, 60000) diff --git a/pom.xml b/pom.xml -index 22922143fc3..97332f7e6ac 100644 +index 22922143fc3..d8eefb0abe2 100644 --- a/pom.xml +++ b/pom.xml @@ -148,6 +148,8 @@ @@ -1763,17 +1763,16 @@ index 47d5ff67b84..8dc8f65d4b1 100644 withSQLConf(SQLConf.ADAPTIVE_EXECUTION_ENABLED.key -> "true", SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> "-1", diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/RemoveRedundantProjectsSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/RemoveRedundantProjectsSuite.scala -index b5bac8079c4..1606dd057e0 100644 +index b5bac8079c4..873d7d95843 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/RemoveRedundantProjectsSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/RemoveRedundantProjectsSuite.scala @@ -17,7 +17,10 @@ package org.apache.spark.sql.execution --import org.apache.spark.sql.{DataFrame, QueryTest, Row} +import org.apache.comet.CometConf + -+import org.apache.spark.sql.{DataFrame, IgnoreComet, QueryTest, Row} + import org.apache.spark.sql.{DataFrame, QueryTest, Row} +import org.apache.spark.sql.comet.CometProjectExec import org.apache.spark.sql.connector.SimpleWritableDataSource import org.apache.spark.sql.execution.adaptive.{AdaptiveSparkPlanHelper, DisableAdaptiveExecutionSuite, EnableAdaptiveExecutionSuite} From 04ff13b4a84667332fd1f635bc4a768e5f928412 Mon Sep 17 00:00:00 2001 From: Matt Butrovich Date: Tue, 21 Apr 2026 12:30:10 -0400 Subject: [PATCH 14/21] Update Spark diffs again for SubquerySuite and other remaining failures. --- dev/diffs/3.4.3.diff | 63 ++++++++++++++++++++++++++++++++------- dev/diffs/3.5.8.diff | 65 ++++++++++++++++++++++++++++++++-------- dev/diffs/4.0.1.diff | 70 +++++++++++++++++++++++++++++++++++--------- 3 files changed, 161 insertions(+), 37 deletions(-) diff --git a/dev/diffs/3.4.3.diff b/dev/diffs/3.4.3.diff index ae9490ddb3..e38a8d90f4 100644 --- a/dev/diffs/3.4.3.diff +++ b/dev/diffs/3.4.3.diff @@ -417,7 +417,7 @@ index daef11ae4d6..9f3cc9181f2 100644 assert(exchanges.size == 2) } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DynamicPartitionPruningSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DynamicPartitionPruningSuite.scala -index f33432ddb6f..579a9c271de 100644 +index f33432ddb6f..99729e465e4 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/DynamicPartitionPruningSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/DynamicPartitionPruningSuite.scala @@ -22,6 +22,7 @@ import org.scalatest.GivenWhenThen @@ -436,7 +436,16 @@ index f33432ddb6f..579a9c271de 100644 } val hasFilter = if (withSubquery) "Should" else "Shouldn't" -@@ -262,6 +264,12 @@ abstract class DynamicPartitionPruningSuiteBase +@@ -247,6 +249,8 @@ abstract class DynamicPartitionPruningSuiteBase + val buf = collectDynamicPruningExpressions(df.queryExecution.executedPlan).collect { + case InSubqueryExec(_, b: SubqueryBroadcastExec, _, _, _, _) => + b.index ++ case InSubqueryExec(_, b: CometSubqueryBroadcastExec, _, _, _, _) => ++ b.indices.head + } + assert(buf.distinct.size == n) + } +@@ -262,6 +266,12 @@ abstract class DynamicPartitionPruningSuiteBase case s: BatchScanExec => s.runtimeFilters.collect { case d: DynamicPruningExpression => d.child } @@ -449,7 +458,7 @@ index f33432ddb6f..579a9c271de 100644 case _ => Nil } } -@@ -1027,7 +1035,8 @@ abstract class DynamicPartitionPruningSuiteBase +@@ -1027,7 +1037,8 @@ abstract class DynamicPartitionPruningSuiteBase } } @@ -459,7 +468,26 @@ index f33432ddb6f..579a9c271de 100644 withSQLConf(SQLConf.DYNAMIC_PARTITION_PRUNING_REUSE_BROADCAST_ONLY.key -> "false", SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> "-1") { withTable("large", "dimTwo", "dimThree") { -@@ -1215,7 +1224,8 @@ abstract class DynamicPartitionPruningSuiteBase +@@ -1204,10 +1215,16 @@ abstract class DynamicPartitionPruningSuiteBase + + val plan = df.queryExecution.executedPlan + val countSubqueryBroadcasts = +- collectWithSubqueries(plan)({ case _: SubqueryBroadcastExec => 1 }).sum ++ collectWithSubqueries(plan)({ ++ case _: SubqueryBroadcastExec => 1 ++ case _: CometSubqueryBroadcastExec => 1 ++ }).sum + + val countReusedSubqueryBroadcasts = +- collectWithSubqueries(plan)({ case ReusedSubqueryExec(_: SubqueryBroadcastExec) => 1}).sum ++ collectWithSubqueries(plan)({ ++ case ReusedSubqueryExec(_: SubqueryBroadcastExec) => 1 ++ case ReusedSubqueryExec(_: CometSubqueryBroadcastExec) => 1 ++ }).sum + + assert(countSubqueryBroadcasts == 1) + assert(countReusedSubqueryBroadcasts == 1) +@@ -1215,7 +1232,8 @@ abstract class DynamicPartitionPruningSuiteBase } test("SPARK-32509: Unused Dynamic Pruning filter shouldn't affect " + @@ -469,7 +497,7 @@ index f33432ddb6f..579a9c271de 100644 withSQLConf(SQLConf.DYNAMIC_PARTITION_PRUNING_REUSE_BROADCAST_ONLY.key -> "true") { withSQLConf(SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> "-1") { val df = sql( -@@ -1423,7 +1433,8 @@ abstract class DynamicPartitionPruningSuiteBase +@@ -1423,7 +1441,8 @@ abstract class DynamicPartitionPruningSuiteBase } } @@ -479,7 +507,15 @@ index f33432ddb6f..579a9c271de 100644 withSQLConf(SQLConf.DYNAMIC_PARTITION_PRUNING_REUSE_BROADCAST_ONLY.key -> "true") { val df = sql( """ WITH v as ( -@@ -1698,7 +1709,8 @@ abstract class DynamicPartitionPruningV1Suite extends DynamicPartitionPruningDat +@@ -1577,6 +1596,7 @@ abstract class DynamicPartitionPruningSuiteBase + + val subqueryBroadcastExecs = collectWithSubqueries(df.queryExecution.executedPlan) { + case s: SubqueryBroadcastExec => s ++ case s: CometSubqueryBroadcastExec => s + } + assert(subqueryBroadcastExecs.size === 1) + subqueryBroadcastExecs.foreach { subqueryBroadcastExec => +@@ -1698,7 +1718,8 @@ abstract class DynamicPartitionPruningV1Suite extends DynamicPartitionPruningDat * Check the static scan metrics with and without DPP */ test("static scan metrics", @@ -489,7 +525,7 @@ index f33432ddb6f..579a9c271de 100644 withSQLConf(SQLConf.DYNAMIC_PARTITION_PRUNING_ENABLED.key -> "true", SQLConf.DYNAMIC_PARTITION_PRUNING_REUSE_BROADCAST_ONLY.key -> "false", SQLConf.EXCHANGE_REUSE_ENABLED.key -> "false") { -@@ -1729,6 +1741,8 @@ abstract class DynamicPartitionPruningV1Suite extends DynamicPartitionPruningDat +@@ -1729,6 +1750,8 @@ abstract class DynamicPartitionPruningV1Suite extends DynamicPartitionPruningDat case s: BatchScanExec => // we use f1 col for v2 tables due to schema pruning s.output.exists(_.exists(_.argString(maxFields = 100).contains("f1"))) @@ -1056,14 +1092,14 @@ index 18123a4d6ec..0fe185baa33 100644 test("non-matching optional group") { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/SubquerySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/SubquerySuite.scala -index 75eabcb96f2..7c0bbd71551 100644 +index 75eabcb96f2..7a681f147e4 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/SubquerySuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/SubquerySuite.scala @@ -21,10 +21,11 @@ import scala.collection.mutable.ArrayBuffer import org.apache.spark.sql.catalyst.expressions.SubqueryExpression import org.apache.spark.sql.catalyst.plans.logical.{Aggregate, Join, LogicalPlan, Project, Sort, Union} -+import org.apache.spark.sql.comet.CometScanExec ++import org.apache.spark.sql.comet.{CometNativeColumnarToRowExec, CometNativeScanExec, CometScanExec} import org.apache.spark.sql.execution._ import org.apache.spark.sql.execution.adaptive.{AdaptiveSparkPlanHelper, DisableAdaptiveExecution} import org.apache.spark.sql.execution.datasources.FileScanRDD @@ -1072,7 +1108,7 @@ index 75eabcb96f2..7c0bbd71551 100644 import org.apache.spark.sql.execution.joins.{BaseJoinExec, BroadcastHashJoinExec, BroadcastNestedLoopJoinExec} import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.test.SharedSparkSession -@@ -1543,6 +1544,12 @@ class SubquerySuite extends QueryTest +@@ -1543,6 +1544,17 @@ class SubquerySuite extends QueryTest fs.inputRDDs().forall( _.asInstanceOf[FileScanRDD].filePartitions.forall( _.files.forall(_.urlEncodedPath.contains("p=0")))) @@ -1081,11 +1117,16 @@ index 75eabcb96f2..7c0bbd71551 100644 + partitionFilters.exists(ExecSubqueryExpression.hasSubquery) && + fs.inputRDDs().forall( + _.asInstanceOf[FileScanRDD].filePartitions.forall( ++ _.files.forall(_.urlEncodedPath.contains("p=0")))) ++ case CometNativeColumnarToRowExec(fs: CometNativeScanExec) => ++ fs.partitionFilters.exists(ExecSubqueryExpression.hasSubquery) && ++ fs.inputRDDs().forall( ++ _.asInstanceOf[FileScanRDD].filePartitions.forall( + _.files.forall(_.urlEncodedPath.contains("p=0")))) case _ => false }) } -@@ -2108,7 +2115,7 @@ class SubquerySuite extends QueryTest +@@ -2108,7 +2120,7 @@ class SubquerySuite extends QueryTest df.collect() val exchanges = collect(df.queryExecution.executedPlan) { diff --git a/dev/diffs/3.5.8.diff b/dev/diffs/3.5.8.diff index 6b83f3a05a..5a8f779f75 100644 --- a/dev/diffs/3.5.8.diff +++ b/dev/diffs/3.5.8.diff @@ -1,5 +1,5 @@ diff --git a/pom.xml b/pom.xml -index edd2ad57880..837b95d1ada 100644 +index edd2ad57880..d5273840330 100644 --- a/pom.xml +++ b/pom.xml @@ -152,6 +152,8 @@ @@ -398,7 +398,7 @@ index c4fb4fa943c..a04b23870a8 100644 assert(exchanges.size == 2) } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DynamicPartitionPruningSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DynamicPartitionPruningSuite.scala -index f33432ddb6f..579a9c271de 100644 +index f33432ddb6f..99729e465e4 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/DynamicPartitionPruningSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/DynamicPartitionPruningSuite.scala @@ -22,6 +22,7 @@ import org.scalatest.GivenWhenThen @@ -417,7 +417,16 @@ index f33432ddb6f..579a9c271de 100644 } val hasFilter = if (withSubquery) "Should" else "Shouldn't" -@@ -262,6 +264,12 @@ abstract class DynamicPartitionPruningSuiteBase +@@ -247,6 +249,8 @@ abstract class DynamicPartitionPruningSuiteBase + val buf = collectDynamicPruningExpressions(df.queryExecution.executedPlan).collect { + case InSubqueryExec(_, b: SubqueryBroadcastExec, _, _, _, _) => + b.index ++ case InSubqueryExec(_, b: CometSubqueryBroadcastExec, _, _, _, _) => ++ b.indices.head + } + assert(buf.distinct.size == n) + } +@@ -262,6 +266,12 @@ abstract class DynamicPartitionPruningSuiteBase case s: BatchScanExec => s.runtimeFilters.collect { case d: DynamicPruningExpression => d.child } @@ -430,7 +439,7 @@ index f33432ddb6f..579a9c271de 100644 case _ => Nil } } -@@ -1027,7 +1035,8 @@ abstract class DynamicPartitionPruningSuiteBase +@@ -1027,7 +1037,8 @@ abstract class DynamicPartitionPruningSuiteBase } } @@ -440,7 +449,26 @@ index f33432ddb6f..579a9c271de 100644 withSQLConf(SQLConf.DYNAMIC_PARTITION_PRUNING_REUSE_BROADCAST_ONLY.key -> "false", SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> "-1") { withTable("large", "dimTwo", "dimThree") { -@@ -1215,7 +1224,8 @@ abstract class DynamicPartitionPruningSuiteBase +@@ -1204,10 +1215,16 @@ abstract class DynamicPartitionPruningSuiteBase + + val plan = df.queryExecution.executedPlan + val countSubqueryBroadcasts = +- collectWithSubqueries(plan)({ case _: SubqueryBroadcastExec => 1 }).sum ++ collectWithSubqueries(plan)({ ++ case _: SubqueryBroadcastExec => 1 ++ case _: CometSubqueryBroadcastExec => 1 ++ }).sum + + val countReusedSubqueryBroadcasts = +- collectWithSubqueries(plan)({ case ReusedSubqueryExec(_: SubqueryBroadcastExec) => 1}).sum ++ collectWithSubqueries(plan)({ ++ case ReusedSubqueryExec(_: SubqueryBroadcastExec) => 1 ++ case ReusedSubqueryExec(_: CometSubqueryBroadcastExec) => 1 ++ }).sum + + assert(countSubqueryBroadcasts == 1) + assert(countReusedSubqueryBroadcasts == 1) +@@ -1215,7 +1232,8 @@ abstract class DynamicPartitionPruningSuiteBase } test("SPARK-32509: Unused Dynamic Pruning filter shouldn't affect " + @@ -450,7 +478,7 @@ index f33432ddb6f..579a9c271de 100644 withSQLConf(SQLConf.DYNAMIC_PARTITION_PRUNING_REUSE_BROADCAST_ONLY.key -> "true") { withSQLConf(SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> "-1") { val df = sql( -@@ -1423,7 +1433,8 @@ abstract class DynamicPartitionPruningSuiteBase +@@ -1423,7 +1441,8 @@ abstract class DynamicPartitionPruningSuiteBase } } @@ -460,7 +488,15 @@ index f33432ddb6f..579a9c271de 100644 withSQLConf(SQLConf.DYNAMIC_PARTITION_PRUNING_REUSE_BROADCAST_ONLY.key -> "true") { val df = sql( """ WITH v as ( -@@ -1698,7 +1709,8 @@ abstract class DynamicPartitionPruningV1Suite extends DynamicPartitionPruningDat +@@ -1577,6 +1596,7 @@ abstract class DynamicPartitionPruningSuiteBase + + val subqueryBroadcastExecs = collectWithSubqueries(df.queryExecution.executedPlan) { + case s: SubqueryBroadcastExec => s ++ case s: CometSubqueryBroadcastExec => s + } + assert(subqueryBroadcastExecs.size === 1) + subqueryBroadcastExecs.foreach { subqueryBroadcastExec => +@@ -1698,7 +1718,8 @@ abstract class DynamicPartitionPruningV1Suite extends DynamicPartitionPruningDat * Check the static scan metrics with and without DPP */ test("static scan metrics", @@ -470,7 +506,7 @@ index f33432ddb6f..579a9c271de 100644 withSQLConf(SQLConf.DYNAMIC_PARTITION_PRUNING_ENABLED.key -> "true", SQLConf.DYNAMIC_PARTITION_PRUNING_REUSE_BROADCAST_ONLY.key -> "false", SQLConf.EXCHANGE_REUSE_ENABLED.key -> "false") { -@@ -1729,6 +1741,8 @@ abstract class DynamicPartitionPruningV1Suite extends DynamicPartitionPruningDat +@@ -1729,6 +1750,8 @@ abstract class DynamicPartitionPruningV1Suite extends DynamicPartitionPruningDat case s: BatchScanExec => // we use f1 col for v2 tables due to schema pruning s.output.exists(_.exists(_.argString(maxFields = 100).contains("f1"))) @@ -984,14 +1020,14 @@ index 8b4ac474f87..3f79f20822f 100644 extensions.injectColumnar(session => MyColumnarRule(PreRuleReplaceAddWithBrokenVersion(), MyPostRule())) } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/SubquerySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/SubquerySuite.scala -index 04702201f82..5ee11f83ecf 100644 +index 04702201f82..a4b5c6c93ce 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/SubquerySuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/SubquerySuite.scala @@ -22,10 +22,11 @@ import scala.collection.mutable.ArrayBuffer import org.apache.spark.sql.catalyst.expressions.SubqueryExpression import org.apache.spark.sql.catalyst.plans.{LeftAnti, LeftSemi} import org.apache.spark.sql.catalyst.plans.logical.{Aggregate, Join, LogicalPlan, Project, Sort, Union} -+import org.apache.spark.sql.comet.CometScanExec ++import org.apache.spark.sql.comet.{CometNativeColumnarToRowExec, CometNativeScanExec, CometScanExec} import org.apache.spark.sql.execution._ import org.apache.spark.sql.execution.adaptive.{AdaptiveSparkPlanHelper, DisableAdaptiveExecution} import org.apache.spark.sql.execution.datasources.FileScanRDD @@ -1000,7 +1036,7 @@ index 04702201f82..5ee11f83ecf 100644 import org.apache.spark.sql.execution.joins.{BaseJoinExec, BroadcastHashJoinExec, BroadcastNestedLoopJoinExec} import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.test.SharedSparkSession -@@ -1599,6 +1600,12 @@ class SubquerySuite extends QueryTest +@@ -1599,6 +1600,17 @@ class SubquerySuite extends QueryTest fs.inputRDDs().forall( _.asInstanceOf[FileScanRDD].filePartitions.forall( _.files.forall(_.urlEncodedPath.contains("p=0")))) @@ -1009,11 +1045,16 @@ index 04702201f82..5ee11f83ecf 100644 + partitionFilters.exists(ExecSubqueryExpression.hasSubquery) && + fs.inputRDDs().forall( + _.asInstanceOf[FileScanRDD].filePartitions.forall( ++ _.files.forall(_.urlEncodedPath.contains("p=0")))) ++ case CometNativeColumnarToRowExec(fs: CometNativeScanExec) => ++ fs.partitionFilters.exists(ExecSubqueryExpression.hasSubquery) && ++ fs.inputRDDs().forall( ++ _.asInstanceOf[FileScanRDD].filePartitions.forall( + _.files.forall(_.urlEncodedPath.contains("p=0")))) case _ => false }) } -@@ -2164,7 +2171,7 @@ class SubquerySuite extends QueryTest +@@ -2164,7 +2176,7 @@ class SubquerySuite extends QueryTest df.collect() val exchanges = collect(df.queryExecution.executedPlan) { diff --git a/dev/diffs/4.0.1.diff b/dev/diffs/4.0.1.diff index 37b72dc05b..d846189720 100644 --- a/dev/diffs/4.0.1.diff +++ b/dev/diffs/4.0.1.diff @@ -589,7 +589,7 @@ index 81713c777bc..b5f92ed9742 100644 assert(exchanges.size == 2) } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DynamicPartitionPruningSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DynamicPartitionPruningSuite.scala -index 2c24cc7d570..fd3c85eef87 100644 +index 2c24cc7d570..e68e44be724 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/DynamicPartitionPruningSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/DynamicPartitionPruningSuite.scala @@ -22,6 +22,7 @@ import org.scalatest.GivenWhenThen @@ -608,7 +608,16 @@ index 2c24cc7d570..fd3c85eef87 100644 } val hasFilter = if (withSubquery) "Should" else "Shouldn't" -@@ -262,6 +264,12 @@ abstract class DynamicPartitionPruningSuiteBase +@@ -247,6 +249,8 @@ abstract class DynamicPartitionPruningSuiteBase + val buf = collectDynamicPruningExpressions(df.queryExecution.executedPlan).collect { + case InSubqueryExec(_, b: SubqueryBroadcastExec, _, _, _, _) => + b.indices.map(idx => b.buildKeys(idx)) ++ case InSubqueryExec(_, b: CometSubqueryBroadcastExec, _, _, _, _) => ++ b.indices.map(idx => b.buildKeys(idx)) + } + assert(buf.distinct.size == n) + } +@@ -262,6 +266,12 @@ abstract class DynamicPartitionPruningSuiteBase case s: BatchScanExec => s.runtimeFilters.collect { case d: DynamicPruningExpression => d.child } @@ -621,7 +630,7 @@ index 2c24cc7d570..fd3c85eef87 100644 case _ => Nil } } -@@ -1027,7 +1035,8 @@ abstract class DynamicPartitionPruningSuiteBase +@@ -1027,7 +1037,8 @@ abstract class DynamicPartitionPruningSuiteBase } } @@ -631,7 +640,7 @@ index 2c24cc7d570..fd3c85eef87 100644 withSQLConf(SQLConf.DYNAMIC_PARTITION_PRUNING_REUSE_BROADCAST_ONLY.key -> "false", SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> "-1") { withTable("large", "dimTwo", "dimThree") { -@@ -1151,7 +1160,8 @@ abstract class DynamicPartitionPruningSuiteBase +@@ -1151,7 +1162,8 @@ abstract class DynamicPartitionPruningSuiteBase } } @@ -641,7 +650,26 @@ index 2c24cc7d570..fd3c85eef87 100644 withSQLConf(SQLConf.DYNAMIC_PARTITION_PRUNING_REUSE_BROADCAST_ONLY.key -> "true", SQLConf.ADAPTIVE_OPTIMIZER_EXCLUDED_RULES.key -> AQEPropagateEmptyRelation.ruleName, SQLConf.ANSI_ENABLED.key -> "false" // ANSI mode doesn't support "String + String" -@@ -1215,7 +1225,8 @@ abstract class DynamicPartitionPruningSuiteBase +@@ -1204,10 +1216,16 @@ abstract class DynamicPartitionPruningSuiteBase + + val plan = df.queryExecution.executedPlan + val countSubqueryBroadcasts = +- collectWithSubqueries(plan)({ case _: SubqueryBroadcastExec => 1 }).sum ++ collectWithSubqueries(plan)({ ++ case _: SubqueryBroadcastExec => 1 ++ case _: CometSubqueryBroadcastExec => 1 ++ }).sum + + val countReusedSubqueryBroadcasts = +- collectWithSubqueries(plan)({ case ReusedSubqueryExec(_: SubqueryBroadcastExec) => 1}).sum ++ collectWithSubqueries(plan)({ ++ case ReusedSubqueryExec(_: SubqueryBroadcastExec) => 1 ++ case ReusedSubqueryExec(_: CometSubqueryBroadcastExec) => 1 ++ }).sum + + assert(countSubqueryBroadcasts == 1) + assert(countReusedSubqueryBroadcasts == 1) +@@ -1215,7 +1233,8 @@ abstract class DynamicPartitionPruningSuiteBase } test("SPARK-32509: Unused Dynamic Pruning filter shouldn't affect " + @@ -651,7 +679,7 @@ index 2c24cc7d570..fd3c85eef87 100644 withSQLConf(SQLConf.DYNAMIC_PARTITION_PRUNING_REUSE_BROADCAST_ONLY.key -> "true") { withSQLConf(SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> "-1") { val df = sql( -@@ -1330,6 +1341,7 @@ abstract class DynamicPartitionPruningSuiteBase +@@ -1330,6 +1349,7 @@ abstract class DynamicPartitionPruningSuiteBase } test("Subquery reuse across the whole plan", @@ -659,7 +687,7 @@ index 2c24cc7d570..fd3c85eef87 100644 DisableAdaptiveExecution("DPP in AQE must reuse broadcast")) { withSQLConf(SQLConf.DYNAMIC_PARTITION_PRUNING_ENABLED.key -> "true", SQLConf.DYNAMIC_PARTITION_PRUNING_REUSE_BROADCAST_ONLY.key -> "false", -@@ -1424,7 +1436,8 @@ abstract class DynamicPartitionPruningSuiteBase +@@ -1424,7 +1444,8 @@ abstract class DynamicPartitionPruningSuiteBase } } @@ -669,7 +697,15 @@ index 2c24cc7d570..fd3c85eef87 100644 withSQLConf(SQLConf.DYNAMIC_PARTITION_PRUNING_REUSE_BROADCAST_ONLY.key -> "true") { val df = sql( """ WITH v as ( -@@ -1699,7 +1712,8 @@ abstract class DynamicPartitionPruningV1Suite extends DynamicPartitionPruningDat +@@ -1578,6 +1599,7 @@ abstract class DynamicPartitionPruningSuiteBase + + val subqueryBroadcastExecs = collectWithSubqueries(df.queryExecution.executedPlan) { + case s: SubqueryBroadcastExec => s ++ case s: CometSubqueryBroadcastExec => s + } + assert(subqueryBroadcastExecs.size === 1) + subqueryBroadcastExecs.foreach { subqueryBroadcastExec => +@@ -1699,7 +1721,8 @@ abstract class DynamicPartitionPruningV1Suite extends DynamicPartitionPruningDat * Check the static scan metrics with and without DPP */ test("static scan metrics", @@ -679,7 +715,7 @@ index 2c24cc7d570..fd3c85eef87 100644 withSQLConf(SQLConf.DYNAMIC_PARTITION_PRUNING_ENABLED.key -> "true", SQLConf.DYNAMIC_PARTITION_PRUNING_REUSE_BROADCAST_ONLY.key -> "false", SQLConf.EXCHANGE_REUSE_ENABLED.key -> "false") { -@@ -1730,6 +1744,8 @@ abstract class DynamicPartitionPruningV1Suite extends DynamicPartitionPruningDat +@@ -1730,6 +1753,8 @@ abstract class DynamicPartitionPruningV1Suite extends DynamicPartitionPruningDat case s: BatchScanExec => // we use f1 col for v2 tables due to schema pruning s.output.exists(_.exists(_.argString(maxFields = 100).contains("f1"))) @@ -1301,14 +1337,14 @@ index 0df7f806272..92390bd819f 100644 test("non-matching optional group") { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/SubquerySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/SubquerySuite.scala -index 2e33f6505ab..c74f3e5562f 100644 +index 2e33f6505ab..6d4a75d02ff 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/SubquerySuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/SubquerySuite.scala @@ -23,12 +23,14 @@ import org.apache.spark.SparkRuntimeException import org.apache.spark.sql.catalyst.expressions.SubqueryExpression import org.apache.spark.sql.catalyst.plans.{LeftAnti, LeftSemi} import org.apache.spark.sql.catalyst.plans.logical.{Aggregate, Filter, Join, LogicalPlan, Project, Sort, Union} -+import org.apache.spark.sql.comet.CometScanExec ++import org.apache.spark.sql.comet.{CometNativeColumnarToRowExec, CometNativeScanExec, CometScanExec} import org.apache.spark.sql.execution._ import org.apache.spark.sql.execution.adaptive.{AdaptiveSparkPlanHelper, DisableAdaptiveExecution} import org.apache.spark.sql.execution.datasources.FileScanRDD @@ -1320,7 +1356,7 @@ index 2e33f6505ab..c74f3e5562f 100644 import org.apache.spark.sql.test.SharedSparkSession class SubquerySuite extends QueryTest -@@ -1529,6 +1531,12 @@ class SubquerySuite extends QueryTest +@@ -1529,6 +1531,18 @@ class SubquerySuite extends QueryTest fs.inputRDDs().forall( _.asInstanceOf[FileScanRDD].filePartitions.forall( _.files.forall(_.urlEncodedPath.contains("p=0")))) @@ -1329,11 +1365,17 @@ index 2e33f6505ab..c74f3e5562f 100644 + partitionFilters.exists(ExecSubqueryExpression.hasSubquery) && + fs.inputRDDs().forall( + _.asInstanceOf[FileScanRDD].filePartitions.forall( ++ _.files.forall(_.urlEncodedPath.contains("p=0")))) ++ case CometNativeColumnarToRowExec( ++ fs: CometNativeScanExec) => ++ fs.partitionFilters.exists(ExecSubqueryExpression.hasSubquery) && ++ fs.inputRDDs().forall( ++ _.asInstanceOf[FileScanRDD].filePartitions.forall( + _.files.forall(_.urlEncodedPath.contains("p=0")))) case _ => false }) } -@@ -2094,7 +2102,7 @@ class SubquerySuite extends QueryTest +@@ -2094,7 +2108,7 @@ class SubquerySuite extends QueryTest df.collect() val exchanges = collect(df.queryExecution.executedPlan) { @@ -1342,7 +1384,7 @@ index 2e33f6505ab..c74f3e5562f 100644 } assert(exchanges.size === 1) } -@@ -2674,22 +2682,31 @@ class SubquerySuite extends QueryTest +@@ -2674,22 +2688,31 @@ class SubquerySuite extends QueryTest } } From 74a452a22e07ce9e3be293ac0fb0155fddf73817 Mon Sep 17 00:00:00 2001 From: Matt Butrovich Date: Tue, 21 Apr 2026 12:43:20 -0400 Subject: [PATCH 15/21] Cleanup. --- .../apache/spark/sql/comet/CometNativeScanExec.scala | 8 +++++--- .../scala/org/apache/comet/exec/CometExecSuite.scala | 10 ++++++++++ 2 files changed, 15 insertions(+), 3 deletions(-) diff --git a/spark/src/main/scala/org/apache/spark/sql/comet/CometNativeScanExec.scala b/spark/src/main/scala/org/apache/spark/sql/comet/CometNativeScanExec.scala index cff9267cb4..84de323f51 100644 --- a/spark/src/main/scala/org/apache/spark/sql/comet/CometNativeScanExec.scala +++ b/spark/src/main/scala/org/apache/spark/sql/comet/CometNativeScanExec.scala @@ -193,9 +193,11 @@ case class CometNativeScanExec( } case _ => } - // Also resolve DPP in CometScanExec's partitionFilters, which may reference - // a different InSubqueryExec instance (with the original SubqueryBroadcastExec). - // CometScanExec.dynamicallySelectedPartitions evaluates these filters. + // CometNativeScanExec.partitionFilters and CometScanExec.partitionFilters contain + // different InSubqueryExec instances. convertSubqueryBroadcasts replaced the former with + // CometSubqueryBroadcastExec, but the latter still has the original SubqueryBroadcastExec. + // Both need resolution because CometScanExec.dynamicallySelectedPartitions evaluates its + // own partitionFilters. updateResult() is a no-op if already resolved. if (scan != null) { scan.partitionFilters.foreach { case DynamicPruningExpression(e: InSubqueryExec) if e.values().isEmpty => diff --git a/spark/src/test/scala/org/apache/comet/exec/CometExecSuite.scala b/spark/src/test/scala/org/apache/comet/exec/CometExecSuite.scala index 08c72cf5c7..999c4e150a 100644 --- a/spark/src/test/scala/org/apache/comet/exec/CometExecSuite.scala +++ b/spark/src/test/scala/org/apache/comet/exec/CometExecSuite.scala @@ -321,6 +321,16 @@ class CometExecSuite extends CometTestBase { assert( broadcasts.size == 1, s"Expected exactly 1 CometBroadcastExchangeExec (other reused):\n${cometPlan.treeString}") + + // Verify canonical forms match — this is what ReuseExchangeAndSubquery uses to + // determine reuse eligibility + if (reused.nonEmpty && broadcasts.nonEmpty) { + val reusedChild = reused.head.child + assert( + reusedChild.canonicalized == broadcasts.head.canonicalized, + s"ReusedExchangeExec child and CometBroadcastExchangeExec should have same " + + s"canonical form for reuse") + } } } } From a98741cb2e1a5f8edc88f78ace7d008a90064b3a Mon Sep 17 00:00:00 2001 From: Matt Butrovich Date: Tue, 21 Apr 2026 12:49:25 -0400 Subject: [PATCH 16/21] fix format. --- .../src/test/scala/org/apache/comet/exec/CometExecSuite.scala | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/spark/src/test/scala/org/apache/comet/exec/CometExecSuite.scala b/spark/src/test/scala/org/apache/comet/exec/CometExecSuite.scala index 999c4e150a..93f3d88220 100644 --- a/spark/src/test/scala/org/apache/comet/exec/CometExecSuite.scala +++ b/spark/src/test/scala/org/apache/comet/exec/CometExecSuite.scala @@ -328,8 +328,8 @@ class CometExecSuite extends CometTestBase { val reusedChild = reused.head.child assert( reusedChild.canonicalized == broadcasts.head.canonicalized, - s"ReusedExchangeExec child and CometBroadcastExchangeExec should have same " + - s"canonical form for reuse") + "ReusedExchangeExec child and CometBroadcastExchangeExec should have same " + + "canonical form for reuse") } } } From 499d2ad51c85b48948b655c7cbbc42250e5b1e03 Mon Sep 17 00:00:00 2001 From: Matt Butrovich Date: Tue, 21 Apr 2026 13:13:22 -0400 Subject: [PATCH 17/21] fix complex type join keys for reusebroadcast hash join. add test. --- .../comet/CometSubqueryBroadcastExec.scala | 40 +++++++++---------- .../apache/comet/exec/CometExecSuite.scala | 36 +++++++++++++++++ 2 files changed, 55 insertions(+), 21 deletions(-) diff --git a/spark/src/main/scala/org/apache/spark/sql/comet/CometSubqueryBroadcastExec.scala b/spark/src/main/scala/org/apache/spark/sql/comet/CometSubqueryBroadcastExec.scala index 226854d18c..09ce0c204c 100644 --- a/spark/src/main/scala/org/apache/spark/sql/comet/CometSubqueryBroadcastExec.scala +++ b/spark/src/main/scala/org/apache/spark/sql/comet/CometSubqueryBroadcastExec.scala @@ -23,6 +23,7 @@ import java.util.concurrent.{Future => JFuture} import scala.concurrent.ExecutionContext import scala.concurrent.duration.Duration +import scala.jdk.CollectionConverters._ import org.apache.spark.rdd.RDD import org.apache.spark.sql.catalyst.InternalRow @@ -95,31 +96,31 @@ case class CometSubqueryBroadcastExec( val broadcasted = child.executeBroadcast[Array[ChunkedByteBuffer]]() val arrowBatches = broadcasted.value - // Decode Arrow batches and extract key column values - val keyIndices = indices.map { idx => + // Decode Arrow batches to rows using the same approach as ColumnarToRowExec: + // batch.rowIterator() + UnsafeProjection handles all types including structs/arrays. + val broadcastOutput = child.output + val toUnsafe = UnsafeProjection.create(broadcastOutput, broadcastOutput) + + // Project key columns from the full broadcast output + val keyExprs = indices.map { idx => val key = buildKeys(idx) - // Find the column index in the broadcast output that matches the build key key match { case attr: Attribute => - child.output.indexWhere(_.exprId == attr.exprId) - case Cast(attr: Attribute, _, _, _) => - child.output.indexWhere(_.exprId == attr.exprId) - case _ => idx + val colIdx = broadcastOutput.indexWhere(_.exprId == attr.exprId) + BoundReference(colIdx, key.dataType, key.nullable) + case Cast(attr: Attribute, dt, tz, ansi) => + val colIdx = broadcastOutput.indexWhere(_.exprId == attr.exprId) + Cast(BoundReference(colIdx, attr.dataType, attr.nullable), dt, tz, ansi) + case _ => + BoundReference(idx, key.dataType, key.nullable) } } + val keyProj = UnsafeProjection.create(keyExprs) val rows = arrowBatches.iterator .flatMap(Utils.decodeBatches(_, this.getClass.getSimpleName)) .flatMap { batch => - val numRows = batch.numRows() - (0 until numRows).iterator.map { rowIdx => - val row = batch.getRow(rowIdx) - val projected = new GenericInternalRow(keyIndices.length) - keyIndices.zipWithIndex.foreach { case (colIdx, outIdx) => - projected.update(outIdx, row.get(colIdx, output(outIdx).dataType)) - } - projected.asInstanceOf[InternalRow].copy() - } + batch.rowIterator().asScala.map(toUnsafe).map(keyProj).map(_.copy()) } .toArray .distinct @@ -127,14 +128,11 @@ case class CometSubqueryBroadcastExec( val beforeBuild = System.nanoTime() longMetric("collectTime") += (beforeBuild - beforeCollect) / 1000000 longMetric("numOutputRows") += rows.length - // Convert to UnsafeRow for consistent size metric and to match SubqueryBroadcastExec - val unsafeProj = UnsafeProjection.create(output.map(_.dataType).toArray) - val unsafeRows = rows.map(r => unsafeProj(r).copy()) - val dataSize = unsafeRows.map(_.getSizeInBytes.toLong).sum + val dataSize = rows.map(_.asInstanceOf[UnsafeRow].getSizeInBytes.toLong).sum longMetric("dataSize") += dataSize SQLMetrics.postDriverMetricUpdates(sparkContext, executionId, metrics.values.toSeq) - unsafeRows.asInstanceOf[Array[InternalRow]] + rows.asInstanceOf[Array[InternalRow]] } } } diff --git a/spark/src/test/scala/org/apache/comet/exec/CometExecSuite.scala b/spark/src/test/scala/org/apache/comet/exec/CometExecSuite.scala index 93f3d88220..662e4ca84e 100644 --- a/spark/src/test/scala/org/apache/comet/exec/CometExecSuite.scala +++ b/spark/src/test/scala/org/apache/comet/exec/CometExecSuite.scala @@ -335,6 +335,42 @@ class CometExecSuite extends CometTestBase { } } + test("non-AQE DPP with non-atomic type (struct/array) join key") { + withTempDir { dir => + val path = s"${dir.getAbsolutePath}/data" + withSQLConf(CometConf.COMET_EXEC_ENABLED.key -> "false") { + spark + .range(100) + .selectExpr( + "cast(id % 10 as int) as store_id", + "cast(id as int) as date_id", + "cast(id * 2 as int) as units_sold") + .write + .partitionBy("store_id") + .parquet(s"$path/fact") + spark + .range(10) + .selectExpr("cast(id as int) as store_id", "cast(id as string) as country") + .write + .parquet(s"$path/dim") + } + + Seq("struct", "array").foreach { dataType => + withSQLConf( + SQLConf.USE_V1_SOURCE_LIST.key -> "parquet", + SQLConf.ADAPTIVE_EXECUTION_ENABLED.key -> "false") { + spark.read.parquet(s"$path/fact").createOrReplaceTempView("fact_nonatomic") + spark.read.parquet(s"$path/dim").createOrReplaceTempView("dim_nonatomic") + val df = spark.sql(s"""SELECT f.date_id, f.store_id FROM fact_nonatomic f + |JOIN dim_nonatomic d + |ON $dataType(f.store_id) = $dataType(d.store_id) + |WHERE d.country = 'DE'""".stripMargin) + checkSparkAnswer(df) + } + } + } + } + test("ShuffleQueryStageExec could be direct child node of CometBroadcastExchangeExec") { withSQLConf(CometConf.COMET_SHUFFLE_MODE.key -> "jvm") { val table = "src" From 0a6f290feb6d3a0bc118bf8be843db70375f1ae0 Mon Sep 17 00:00:00 2001 From: Matt Butrovich Date: Tue, 21 Apr 2026 15:47:13 -0400 Subject: [PATCH 18/21] fix equals() operator for CometNativeScanExec --- .../spark/sql/comet/CometNativeScanExec.scala | 3 +- .../apache/comet/exec/CometExecSuite.scala | 171 ++++++++++++++++++ 2 files changed, 173 insertions(+), 1 deletion(-) diff --git a/spark/src/main/scala/org/apache/spark/sql/comet/CometNativeScanExec.scala b/spark/src/main/scala/org/apache/spark/sql/comet/CometNativeScanExec.scala index 84de323f51..cd09f07138 100644 --- a/spark/src/main/scala/org/apache/spark/sql/comet/CometNativeScanExec.scala +++ b/spark/src/main/scala/org/apache/spark/sql/comet/CometNativeScanExec.scala @@ -298,7 +298,8 @@ case class CometNativeScanExec( obj match { case other: CometNativeScanExec => this.originalPlan == other.originalPlan && - this.serializedPlanOpt == other.serializedPlanOpt + this.serializedPlanOpt == other.serializedPlanOpt && + this.partitionFilters == other.partitionFilters case _ => false } diff --git a/spark/src/test/scala/org/apache/comet/exec/CometExecSuite.scala b/spark/src/test/scala/org/apache/comet/exec/CometExecSuite.scala index 662e4ca84e..f073854b2d 100644 --- a/spark/src/test/scala/org/apache/comet/exec/CometExecSuite.scala +++ b/spark/src/test/scala/org/apache/comet/exec/CometExecSuite.scala @@ -371,6 +371,177 @@ class CometExecSuite extends CometTestBase { } } + // Regression tests for DPP exchange/subquery reuse (from DynamicPartitionPruningSuite) + + private def withDppTables(f: => Unit): Unit = { + val factData = Seq( + (1000, 1, 1, 10), + (1010, 2, 1, 10), + (1020, 2, 1, 10), + (1030, 3, 2, 10), + (1040, 3, 2, 50), + (1050, 3, 2, 50), + (1060, 3, 2, 50), + (1070, 4, 2, 10), + (1080, 4, 3, 20), + (1090, 4, 3, 10), + (1100, 4, 3, 10), + (1110, 5, 3, 10), + (1120, 6, 4, 10), + (1130, 7, 4, 50), + (1140, 8, 4, 50), + (1150, 9, 1, 20), + (1160, 10, 1, 20), + (1170, 11, 1, 30), + (1180, 12, 2, 20), + (1190, 13, 2, 20), + (1200, 14, 3, 40), + (1200, 15, 3, 70), + (1210, 16, 4, 10), + (1220, 17, 4, 20), + (1230, 18, 4, 20), + (1240, 19, 5, 40), + (1250, 20, 5, 40), + (1260, 21, 5, 40), + (1270, 22, 5, 50), + (1280, 23, 1, 50), + (1290, 24, 1, 50), + (1300, 25, 1, 50)) + + val storeData = Seq( + (1, "North-Holland", "NL"), + (2, "South-Holland", "NL"), + (3, "Bavaria", "DE"), + (4, "California", "US"), + (5, "Texas", "US"), + (6, "Texas", "US")) + + val storeCode = Seq((1, 10), (2, 20), (3, 30), (4, 40), (5, 50), (6, 60)) + + import testImplicits._ + + withTable("fact_np", "fact_sk", "fact_stats", "dim_stats", "dim_store", "code_stats") { + factData + .toDF("date_id", "store_id", "product_id", "units_sold") + .write + .format("parquet") + .saveAsTable("fact_np") + factData + .toDF("date_id", "store_id", "product_id", "units_sold") + .write + .partitionBy("store_id") + .format("parquet") + .saveAsTable("fact_sk") + factData + .toDF("date_id", "store_id", "product_id", "units_sold") + .write + .partitionBy("store_id") + .format("parquet") + .saveAsTable("fact_stats") + storeData + .toDF("store_id", "state_province", "country") + .write + .format("parquet") + .saveAsTable("dim_store") + storeData + .toDF("store_id", "state_province", "country") + .write + .format("parquet") + .saveAsTable("dim_stats") + storeCode + .toDF("store_id", "code") + .write + .partitionBy("store_id") + .format("parquet") + .saveAsTable("code_stats") + sql("ANALYZE TABLE fact_stats COMPUTE STATISTICS FOR COLUMNS store_id") + sql("ANALYZE TABLE dim_stats COMPUTE STATISTICS FOR COLUMNS store_id") + sql("ANALYZE TABLE dim_store COMPUTE STATISTICS FOR COLUMNS store_id") + sql("ANALYZE TABLE code_stats COMPUTE STATISTICS FOR COLUMNS store_id") + + f + } + } + + test("DPP broadcast exchange reuse") { + withDppTables { + withSQLConf( + SQLConf.USE_V1_SOURCE_LIST.key -> "parquet", + SQLConf.ADAPTIVE_EXECUTION_ENABLED.key -> "false", + SQLConf.DYNAMIC_PARTITION_PRUNING_REUSE_BROADCAST_ONLY.key -> "true") { + + val df = sql("""SELECT /*+ BROADCAST(f)*/ + |f.date_id, f.store_id, f.product_id, f.units_sold FROM fact_np f + |JOIN code_stats s + |ON f.store_id = s.store_id WHERE f.date_id <= 1030""".stripMargin) + val (_, cometPlan) = checkSparkAnswer(df) + + val reusedExchanges = collectWithSubqueries(cometPlan) { case e: ReusedExchangeExec => + e + } + assert( + reusedExchanges.nonEmpty, + s"Expected ReusedExchangeExec for broadcast exchange reuse:\n${cometPlan.treeString}") + } + } + } + + test("DPP subquery reuse with uncorrelated scalar subquery") { + withDppTables { + withSQLConf( + SQLConf.USE_V1_SOURCE_LIST.key -> "parquet", + SQLConf.ADAPTIVE_EXECUTION_ENABLED.key -> "false", + SQLConf.DYNAMIC_PARTITION_PRUNING_REUSE_BROADCAST_ONLY.key -> "true") { + + val df = sql("""SELECT d.store_id, SUM(f.units_sold), + | (SELECT SUM(f.units_sold) + | FROM fact_stats f JOIN dim_stats d ON d.store_id = f.store_id + | WHERE d.country = 'US') AS total_prod + |FROM fact_stats f JOIN dim_stats d ON d.store_id = f.store_id + |WHERE d.country = 'US' + |GROUP BY 1""".stripMargin) + val (_, cometPlan) = checkSparkAnswer(df) + + val countSubqueryBroadcasts = collectWithSubqueries(cometPlan)({ + case _: SubqueryBroadcastExec => 1 + case _: CometSubqueryBroadcastExec => 1 + }).sum + val countReusedSubqueryBroadcasts = collectWithSubqueries(cometPlan)({ + case ReusedSubqueryExec(_: SubqueryBroadcastExec) => 1 + case ReusedSubqueryExec(_: CometSubqueryBroadcastExec) => 1 + }).sum + + assert( + countSubqueryBroadcasts == 1, + s"Expected 1 subquery broadcast but got $countSubqueryBroadcasts:\n" + + cometPlan.treeString) + assert( + countReusedSubqueryBroadcasts == 1, + s"Expected 1 reused subquery broadcast but got $countReusedSubqueryBroadcasts:\n" + + cometPlan.treeString) + } + } + } + + test("DPP with non-atomic type (struct/array) join key") { + withDppTables { + withSQLConf( + SQLConf.USE_V1_SOURCE_LIST.key -> "parquet", + SQLConf.ADAPTIVE_EXECUTION_ENABLED.key -> "false", + SQLConf.DYNAMIC_PARTITION_PRUNING_ENABLED.key -> "true") { + + Seq("struct", "array").foreach { dataType => + val df = + sql(s"""SELECT f.date_id, f.product_id, f.units_sold, f.store_id FROM fact_stats f + |JOIN dim_stats s + |ON $dataType(f.store_id) = $dataType(s.store_id) WHERE s.country = 'DE' + """.stripMargin) + checkSparkAnswer(df) + } + } + } + } + test("ShuffleQueryStageExec could be direct child node of CometBroadcastExchangeExec") { withSQLConf(CometConf.COMET_SHUFFLE_MODE.key -> "jvm") { val table = "src" From 0d1ddd5f1c38b93fa64c90e6c99f94fd3a927a38 Mon Sep 17 00:00:00 2001 From: Matt Butrovich Date: Tue, 21 Apr 2026 16:21:36 -0400 Subject: [PATCH 19/21] fix SPARK-32659: Fix the data issue when pruning DPP on non-atomic type --- .../org/apache/spark/sql/comet/CometSubqueryBroadcastExec.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/spark/src/main/scala/org/apache/spark/sql/comet/CometSubqueryBroadcastExec.scala b/spark/src/main/scala/org/apache/spark/sql/comet/CometSubqueryBroadcastExec.scala index 09ce0c204c..2e27984dbe 100644 --- a/spark/src/main/scala/org/apache/spark/sql/comet/CometSubqueryBroadcastExec.scala +++ b/spark/src/main/scala/org/apache/spark/sql/comet/CometSubqueryBroadcastExec.scala @@ -112,7 +112,7 @@ case class CometSubqueryBroadcastExec( val colIdx = broadcastOutput.indexWhere(_.exprId == attr.exprId) Cast(BoundReference(colIdx, attr.dataType, attr.nullable), dt, tz, ansi) case _ => - BoundReference(idx, key.dataType, key.nullable) + BindReferences.bindReference(key, broadcastOutput) } } val keyProj = UnsafeProjection.create(keyExprs) From ebb096afe326606d23da3890c513d2edb70234d1 Mon Sep 17 00:00:00 2001 From: Matt Butrovich Date: Tue, 21 Apr 2026 16:27:14 -0400 Subject: [PATCH 20/21] strengthen tests. --- .../apache/comet/exec/CometExecSuite.scala | 28 +++++++++++++++++++ 1 file changed, 28 insertions(+) diff --git a/spark/src/test/scala/org/apache/comet/exec/CometExecSuite.scala b/spark/src/test/scala/org/apache/comet/exec/CometExecSuite.scala index f073854b2d..9790c87147 100644 --- a/spark/src/test/scala/org/apache/comet/exec/CometExecSuite.scala +++ b/spark/src/test/scala/org/apache/comet/exec/CometExecSuite.scala @@ -542,6 +542,34 @@ class CometExecSuite extends CometTestBase { } } + test("DPP non-atomic type uses CometSubqueryBroadcastExec") { + withDppTables { + withSQLConf( + SQLConf.USE_V1_SOURCE_LIST.key -> "parquet", + SQLConf.ADAPTIVE_EXECUTION_ENABLED.key -> "false", + SQLConf.DYNAMIC_PARTITION_PRUNING_ENABLED.key -> "true", + SQLConf.DYNAMIC_PARTITION_PRUNING_REUSE_BROADCAST_ONLY.key -> "true") { + + Seq("struct", "array").foreach { dataType => + val df = + sql(s"""SELECT f.date_id, f.product_id, f.units_sold, f.store_id FROM fact_stats f + |JOIN dim_stats s + |ON $dataType(f.store_id) = $dataType(s.store_id) WHERE s.country = 'DE' + """.stripMargin) + val (_, cometPlan) = checkSparkAnswer(df) + + val cometSubqueries = collectWithSubqueries(cometPlan) { + case s: CometSubqueryBroadcastExec => s + } + assert( + cometSubqueries.nonEmpty, + s"Expected DPP with CometSubqueryBroadcastExec for $dataType key:\n" + + cometPlan.treeString) + } + } + } + } + test("ShuffleQueryStageExec could be direct child node of CometBroadcastExchangeExec") { withSQLConf(CometConf.COMET_SHUFFLE_MODE.key -> "jvm") { val table = "src" From 87c7b90f021db5093cb464f19021b293e4011124 Mon Sep 17 00:00:00 2001 From: Matt Butrovich Date: Tue, 21 Apr 2026 16:50:45 -0400 Subject: [PATCH 21/21] Update golden plans to reflect reuse bug fix. --- .../q1.native_datafusion/extended.txt | 8 +- .../q10.native_datafusion/extended.txt | 14 +- .../q11.native_datafusion/extended.txt | 12 +- .../q14a.native_datafusion/extended.txt | 104 +--- .../q14b.native_datafusion/extended.txt | 68 +-- .../q17.native_datafusion/extended.txt | 8 +- .../q23a.native_datafusion/extended.txt | 8 +- .../q23b.native_datafusion/extended.txt | 8 +- .../q25.native_datafusion/extended.txt | 8 +- .../q30.native_datafusion/extended.txt | 8 +- .../q31.native_datafusion/extended.txt | 17 +- .../q32.native_datafusion/extended.txt | 8 +- .../q33.native_datafusion/extended.txt | 14 +- .../q35.native_datafusion/extended.txt | 14 +- .../q38.native_datafusion/extended.txt | 14 +- .../q4.native_datafusion/extended.txt | 22 +- .../q49.native_datafusion/extended.txt | 14 +- .../q5.native_datafusion/extended.txt | 32 +- .../q51.native_datafusion/extended.txt | 8 +- .../q54.native_datafusion/extended.txt | 8 +- .../q56.native_datafusion/extended.txt | 14 +- .../q58.native_datafusion/extended.txt | 34 +- .../q60.native_datafusion/extended.txt | 14 +- .../q61.native_datafusion/extended.txt | 8 +- .../q65.native_datafusion/extended.txt | 8 +- .../q66.native_datafusion/extended.txt | 7 +- .../q69.native_datafusion/extended.txt | 14 +- .../q70.native_datafusion/extended.txt | 8 +- .../q71.native_datafusion/extended.txt | 14 +- .../q74.native_datafusion/extended.txt | 12 +- .../q75.native_datafusion/extended.txt | 22 +- .../q77.native_datafusion/extended.txt | 32 +- .../q78.native_datafusion/extended.txt | 12 +- .../q80.native_datafusion/extended.txt | 14 +- .../q81.native_datafusion/extended.txt | 8 +- .../q83.native_datafusion/extended.txt | 32 +- .../q87.native_datafusion/extended.txt | 14 +- .../q92.native_datafusion/extended.txt | 8 +- .../q97.native_datafusion/extended.txt | 8 +- .../q1.native_datafusion/extended.txt | 8 +- .../q10.native_datafusion/extended.txt | 14 +- .../q11.native_datafusion/extended.txt | 12 +- .../q14a.native_datafusion/extended.txt | 104 +--- .../q14b.native_datafusion/extended.txt | 80 +-- .../q17.native_datafusion/extended.txt | 8 +- .../q23a.native_datafusion/extended.txt | 8 +- .../q23b.native_datafusion/extended.txt | 8 +- .../q25.native_datafusion/extended.txt | 8 +- .../q30.native_datafusion/extended.txt | 8 +- .../q31.native_datafusion/extended.txt | 17 +- .../q32.native_datafusion/extended.txt | 8 +- .../q33.native_datafusion/extended.txt | 14 +- .../q35.native_datafusion/extended.txt | 14 +- .../q38.native_datafusion/extended.txt | 14 +- .../q4.native_datafusion/extended.txt | 22 +- .../q49.native_datafusion/extended.txt | 14 +- .../q5.native_datafusion/extended.txt | 32 +- .../q51.native_datafusion/extended.txt | 8 +- .../q54.native_datafusion/extended.txt | 26 +- .../q56.native_datafusion/extended.txt | 14 +- .../q58.native_datafusion/extended.txt | 50 +- .../q6.native_datafusion/extended.txt | 11 +- .../q60.native_datafusion/extended.txt | 14 +- .../q61.native_datafusion/extended.txt | 8 +- .../q65.native_datafusion/extended.txt | 8 +- .../q66.native_datafusion/extended.txt | 7 +- .../q69.native_datafusion/extended.txt | 14 +- .../q70.native_datafusion/extended.txt | 8 +- .../q71.native_datafusion/extended.txt | 14 +- .../q74.native_datafusion/extended.txt | 12 +- .../q75.native_datafusion/extended.txt | 22 +- .../q77.native_datafusion/extended.txt | 32 +- .../q78.native_datafusion/extended.txt | 12 +- .../q80.native_datafusion/extended.txt | 14 +- .../q81.native_datafusion/extended.txt | 8 +- .../q83.ansi.native_datafusion/extended.txt | 32 +- .../q87.native_datafusion/extended.txt | 14 +- .../q92.native_datafusion/extended.txt | 8 +- .../q97.native_datafusion/extended.txt | 8 +- .../q1.native_datafusion/extended.txt | 8 +- .../q10.native_datafusion/extended.txt | 14 +- .../q11.native_datafusion/extended.txt | 12 +- .../q14a.native_datafusion/extended.txt | 104 +--- .../q14b.native_datafusion/extended.txt | 68 +-- .../q17.native_datafusion/extended.txt | 8 +- .../q23a.native_datafusion/extended.txt | 8 +- .../q23b.native_datafusion/extended.txt | 8 +- .../q25.native_datafusion/extended.txt | 8 +- .../q30.native_datafusion/extended.txt | 8 +- .../q31.native_datafusion/extended.txt | 17 +- .../q32.native_datafusion/extended.txt | 8 +- .../q33.native_datafusion/extended.txt | 14 +- .../q35.native_datafusion/extended.txt | 14 +- .../q38.native_datafusion/extended.txt | 14 +- .../q4.native_datafusion/extended.txt | 22 +- .../q49.native_datafusion/extended.txt | 14 +- .../q5.native_datafusion/extended.txt | 32 +- .../q51.native_datafusion/extended.txt | 8 +- .../q54.native_datafusion/extended.txt | 8 +- .../q56.native_datafusion/extended.txt | 14 +- .../q58.native_datafusion/extended.txt | 24 +- .../q60.native_datafusion/extended.txt | 14 +- .../q61.native_datafusion/extended.txt | 8 +- .../q65.native_datafusion/extended.txt | 8 +- .../q66.native_datafusion/extended.txt | 7 +- .../q69.native_datafusion/extended.txt | 14 +- .../q70.native_datafusion/extended.txt | 8 +- .../q71.native_datafusion/extended.txt | 14 +- .../q74.native_datafusion/extended.txt | 12 +- .../q75.native_datafusion/extended.txt | 22 +- .../q77.native_datafusion/extended.txt | 32 +- .../q78.native_datafusion/extended.txt | 12 +- .../q80.native_datafusion/extended.txt | 14 +- .../q81.native_datafusion/extended.txt | 8 +- .../q83.native_datafusion/extended.txt | 17 +- .../q87.native_datafusion/extended.txt | 14 +- .../q92.native_datafusion/extended.txt | 8 +- .../q97.native_datafusion/extended.txt | 8 +- .../q10a.native_datafusion/extended.txt | 14 +- .../q11.native_datafusion/extended.txt | 12 +- .../q14.native_datafusion/extended.txt | 68 +-- .../q14a.native_datafusion/extended.txt | 482 +++--------------- .../q18a.native_datafusion/extended.txt | 26 +- .../q27a.native_datafusion/extended.txt | 14 +- .../q35.native_datafusion/extended.txt | 14 +- .../q35a.native_datafusion/extended.txt | 14 +- .../q49.native_datafusion/extended.txt | 14 +- .../q51a.native_datafusion/extended.txt | 26 +- .../q5a.native_datafusion/extended.txt | 92 +--- .../q70a.native_datafusion/extended.txt | 20 +- .../q74.native_datafusion/extended.txt | 12 +- .../q75.native_datafusion/extended.txt | 22 +- .../q77a.native_datafusion/extended.txt | 92 +--- .../q78.native_datafusion/extended.txt | 12 +- .../q80a.native_datafusion/extended.txt | 38 +- .../q10a.native_datafusion/extended.txt | 14 +- .../q11.native_datafusion/extended.txt | 12 +- .../q14.native_datafusion/extended.txt | 80 +-- .../q14a.native_datafusion/extended.txt | 482 +++--------------- .../q18a.native_datafusion/extended.txt | 26 +- .../q27a.native_datafusion/extended.txt | 14 +- .../q35.native_datafusion/extended.txt | 14 +- .../q35a.native_datafusion/extended.txt | 14 +- .../q49.native_datafusion/extended.txt | 14 +- .../q51a.native_datafusion/extended.txt | 26 +- .../q5a.native_datafusion/extended.txt | 92 +--- .../q6.native_datafusion/extended.txt | 11 +- .../q70a.native_datafusion/extended.txt | 20 +- .../q74.native_datafusion/extended.txt | 12 +- .../q75.native_datafusion/extended.txt | 22 +- .../q77a.native_datafusion/extended.txt | 92 +--- .../q78.native_datafusion/extended.txt | 12 +- .../q80a.native_datafusion/extended.txt | 38 +- .../q10a.native_datafusion/extended.txt | 14 +- .../q11.native_datafusion/extended.txt | 12 +- .../q14.native_datafusion/extended.txt | 68 +-- .../q14a.native_datafusion/extended.txt | 482 +++--------------- .../q18a.native_datafusion/extended.txt | 26 +- .../q27a.native_datafusion/extended.txt | 14 +- .../q35.native_datafusion/extended.txt | 14 +- .../q35a.native_datafusion/extended.txt | 14 +- .../q49.native_datafusion/extended.txt | 14 +- .../q51a.native_datafusion/extended.txt | 26 +- .../q5a.native_datafusion/extended.txt | 92 +--- .../q70a.native_datafusion/extended.txt | 20 +- .../q74.native_datafusion/extended.txt | 12 +- .../q75.native_datafusion/extended.txt | 22 +- .../q77a.native_datafusion/extended.txt | 92 +--- .../q78.native_datafusion/extended.txt | 12 +- .../q80a.native_datafusion/extended.txt | 38 +- 170 files changed, 943 insertions(+), 4068 deletions(-) diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q1.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q1.native_datafusion/extended.txt index 5875fa3a5e..f49109dafb 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q1.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q1.native_datafusion/extended.txt @@ -35,11 +35,7 @@ CometNativeColumnarToRow : : +- CometBroadcastHashJoin : : :- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.store_returns - : : : +- CometSubqueryBroadcast - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- ReusedSubquery : : +- CometBroadcastExchange : : +- CometProject : : +- CometFilter @@ -53,4 +49,4 @@ CometNativeColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.customer -Comet accelerated 51 out of 53 eligible operators (96%). Final plan contains 1 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 47 out of 49 eligible operators (95%). Final plan contains 1 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q10.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q10.native_datafusion/extended.txt index 2e79f6f464..b7c8209556 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q10.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q10.native_datafusion/extended.txt @@ -33,11 +33,7 @@ TakeOrderedAndProject : : : +- CometProject : : : +- CometBroadcastHashJoin : : : :- CometNativeScan parquet spark_catalog.default.web_sales - : : : : +- CometSubqueryBroadcast - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- ReusedSubquery : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter @@ -47,11 +43,7 @@ TakeOrderedAndProject : : +- CometProject : : +- CometBroadcastHashJoin : : :- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : +- CometSubqueryBroadcast - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- ReusedSubquery : : +- CometBroadcastExchange : : +- CometProject : : +- CometFilter @@ -67,4 +59,4 @@ TakeOrderedAndProject +- CometFilter +- CometNativeScan parquet spark_catalog.default.customer_demographics -Comet accelerated 44 out of 62 eligible operators (70%). Final plan contains 6 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 36 out of 54 eligible operators (66%). Final plan contains 6 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q11.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q11.native_datafusion/extended.txt index edda694b73..7f06caa2e1 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q11.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q11.native_datafusion/extended.txt @@ -63,10 +63,7 @@ CometNativeColumnarToRow : : +- CometBroadcastExchange : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : +- CometSubqueryBroadcast - : : +- CometBroadcastExchange - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- ReusedSubquery : +- CometBroadcastExchange : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -84,12 +81,9 @@ CometNativeColumnarToRow : +- CometBroadcastExchange : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.web_sales - : +- CometSubqueryBroadcast - : +- CometBroadcastExchange - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- ReusedSubquery +- CometBroadcastExchange +- CometFilter +- CometNativeScan parquet spark_catalog.default.date_dim -Comet accelerated 88 out of 92 eligible operators (95%). Final plan contains 1 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 82 out of 86 eligible operators (95%). Final plan contains 1 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q14a.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q14a.native_datafusion/extended.txt index d2a44e9c52..1ae3f23b40 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q14a.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q14a.native_datafusion/extended.txt @@ -16,11 +16,7 @@ CometNativeColumnarToRow : : :- CometProject : : : +- CometBroadcastHashJoin : : : :- CometNativeScan parquet spark_catalog.default.store_sales - : : : : +- CometSubqueryBroadcast - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- ReusedSubquery : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter @@ -28,11 +24,7 @@ CometNativeColumnarToRow : : :- CometProject : : : +- CometBroadcastHashJoin : : : :- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : +- CometSubqueryBroadcast - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- ReusedSubquery : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter @@ -40,11 +32,7 @@ CometNativeColumnarToRow : : +- CometProject : : +- CometBroadcastHashJoin : : :- CometNativeScan parquet spark_catalog.default.web_sales - : : : +- CometSubqueryBroadcast - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- ReusedSubquery : : +- CometBroadcastExchange : : +- CometProject : : +- CometFilter @@ -96,11 +84,7 @@ CometNativeColumnarToRow : : : : : : +- CometBroadcastHashJoin : : : : : : :- CometFilter : : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : : : : +- CometSubqueryBroadcast - : : : : : : : +- CometBroadcastExchange - : : : : : : : +- CometProject - : : : : : : : +- CometFilter - : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : : +- ReusedSubquery : : : : : : +- CometBroadcastExchange : : : : : : +- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.item @@ -119,11 +103,7 @@ CometNativeColumnarToRow : : : : +- CometBroadcastHashJoin : : : : :- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : : : +- CometSubqueryBroadcast - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- ReusedSubquery : : : : +- CometBroadcastExchange : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.item @@ -167,11 +147,7 @@ CometNativeColumnarToRow : : : : : +- CometBroadcastHashJoin : : : : : :- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : : : +- CometSubqueryBroadcast - : : : : : : +- CometBroadcastExchange - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : +- ReusedSubquery : : : : : +- CometBroadcastExchange : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.item @@ -190,11 +166,7 @@ CometNativeColumnarToRow : : : +- CometBroadcastHashJoin : : : :- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : : +- CometSubqueryBroadcast - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- ReusedSubquery : : : +- CometBroadcastExchange : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.item @@ -219,11 +191,7 @@ CometNativeColumnarToRow : : :- CometBroadcastHashJoin : : : :- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : +- CometSubqueryBroadcast - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- ReusedSubquery : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometBroadcastHashJoin @@ -256,11 +224,7 @@ CometNativeColumnarToRow : : : : : : +- CometBroadcastHashJoin : : : : : : :- CometFilter : : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : : : : +- CometSubqueryBroadcast - : : : : : : : +- CometBroadcastExchange - : : : : : : : +- CometProject - : : : : : : : +- CometFilter - : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : : +- ReusedSubquery : : : : : : +- CometBroadcastExchange : : : : : : +- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.item @@ -279,11 +243,7 @@ CometNativeColumnarToRow : : : : +- CometBroadcastHashJoin : : : : :- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : : : +- CometSubqueryBroadcast - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- ReusedSubquery : : : : +- CometBroadcastExchange : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.item @@ -327,11 +287,7 @@ CometNativeColumnarToRow : : : : : +- CometBroadcastHashJoin : : : : : :- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : : : +- CometSubqueryBroadcast - : : : : : : +- CometBroadcastExchange - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : +- ReusedSubquery : : : : : +- CometBroadcastExchange : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.item @@ -350,11 +306,7 @@ CometNativeColumnarToRow : : : +- CometBroadcastHashJoin : : : :- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : : +- CometSubqueryBroadcast - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- ReusedSubquery : : : +- CometBroadcastExchange : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.item @@ -379,11 +331,7 @@ CometNativeColumnarToRow : :- CometBroadcastHashJoin : : :- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : +- CometSubqueryBroadcast - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- ReusedSubquery : : +- CometBroadcastExchange : : +- CometProject : : +- CometBroadcastHashJoin @@ -416,11 +364,7 @@ CometNativeColumnarToRow : : : : : +- CometBroadcastHashJoin : : : : : :- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : : : +- CometSubqueryBroadcast - : : : : : : +- CometBroadcastExchange - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : +- ReusedSubquery : : : : : +- CometBroadcastExchange : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.item @@ -439,11 +383,7 @@ CometNativeColumnarToRow : : : +- CometBroadcastHashJoin : : : :- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : : +- CometSubqueryBroadcast - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- ReusedSubquery : : : +- CometBroadcastExchange : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.item @@ -487,11 +427,7 @@ CometNativeColumnarToRow : : : : +- CometBroadcastHashJoin : : : : :- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : : +- CometSubqueryBroadcast - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- ReusedSubquery : : : : +- CometBroadcastExchange : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.item @@ -510,11 +446,7 @@ CometNativeColumnarToRow : : +- CometBroadcastHashJoin : : :- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : +- CometSubqueryBroadcast - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- ReusedSubquery : : +- CometBroadcastExchange : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.item @@ -527,4 +459,4 @@ CometNativeColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.date_dim -Comet accelerated 499 out of 526 eligible operators (94%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 431 out of 458 eligible operators (94%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q14b.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q14b.native_datafusion/extended.txt index 2d20c40e4b..1b9b46662f 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q14b.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q14b.native_datafusion/extended.txt @@ -11,11 +11,7 @@ CometNativeColumnarToRow : : :- CometProject : : : +- CometBroadcastHashJoin : : : :- CometNativeScan parquet spark_catalog.default.store_sales - : : : : +- CometSubqueryBroadcast - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- ReusedSubquery : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter @@ -23,11 +19,7 @@ CometNativeColumnarToRow : : :- CometProject : : : +- CometBroadcastHashJoin : : : :- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : +- CometSubqueryBroadcast - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- ReusedSubquery : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter @@ -35,11 +27,7 @@ CometNativeColumnarToRow : : +- CometProject : : +- CometBroadcastHashJoin : : :- CometNativeScan parquet spark_catalog.default.web_sales - : : : +- CometSubqueryBroadcast - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- ReusedSubquery : : +- CometBroadcastExchange : : +- CometProject : : +- CometFilter @@ -96,11 +84,7 @@ CometNativeColumnarToRow : : : : : : +- CometBroadcastHashJoin : : : : : : :- CometFilter : : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : : : : +- CometSubqueryBroadcast - : : : : : : : +- CometBroadcastExchange - : : : : : : : +- CometProject - : : : : : : : +- CometFilter - : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : : +- ReusedSubquery : : : : : : +- CometBroadcastExchange : : : : : : +- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.item @@ -119,11 +103,7 @@ CometNativeColumnarToRow : : : : +- CometBroadcastHashJoin : : : : :- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : : : +- CometSubqueryBroadcast - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- ReusedSubquery : : : : +- CometBroadcastExchange : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.item @@ -167,11 +147,7 @@ CometNativeColumnarToRow : : : : : +- CometBroadcastHashJoin : : : : : :- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : : : +- CometSubqueryBroadcast - : : : : : : +- CometBroadcastExchange - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : +- ReusedSubquery : : : : : +- CometBroadcastExchange : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.item @@ -190,11 +166,7 @@ CometNativeColumnarToRow : : : +- CometBroadcastHashJoin : : : :- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : : +- CometSubqueryBroadcast - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- ReusedSubquery : : : +- CometBroadcastExchange : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.item @@ -266,11 +238,7 @@ CometNativeColumnarToRow : : : : : +- CometBroadcastHashJoin : : : : : :- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : : : +- CometSubqueryBroadcast - : : : : : : +- CometBroadcastExchange - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : +- ReusedSubquery : : : : : +- CometBroadcastExchange : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.item @@ -289,11 +257,7 @@ CometNativeColumnarToRow : : : +- CometBroadcastHashJoin : : : :- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : : +- CometSubqueryBroadcast - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- ReusedSubquery : : : +- CometBroadcastExchange : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.item @@ -337,11 +301,7 @@ CometNativeColumnarToRow : : : : +- CometBroadcastHashJoin : : : : :- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : : +- CometSubqueryBroadcast - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- ReusedSubquery : : : : +- CometBroadcastExchange : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.item @@ -360,11 +320,7 @@ CometNativeColumnarToRow : : +- CometBroadcastHashJoin : : :- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : +- CometSubqueryBroadcast - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- ReusedSubquery : : +- CometBroadcastExchange : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.item @@ -382,4 +338,4 @@ CometNativeColumnarToRow : +- CometNativeScan parquet spark_catalog.default.date_dim +- CometNativeScan parquet spark_catalog.default.date_dim -Comet accelerated 354 out of 377 eligible operators (93%). Final plan contains 6 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 310 out of 333 eligible operators (93%). Final plan contains 6 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q17.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q17.native_datafusion/extended.txt index 2650cd8f1a..a3f43554cf 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q17.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q17.native_datafusion/extended.txt @@ -35,11 +35,7 @@ CometNativeColumnarToRow : : : : : +- CometBroadcastExchange : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : : +- CometSubqueryBroadcast - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- ReusedSubquery : : : : +- CometBroadcastExchange : : : : +- CometProject : : : : +- CometFilter @@ -61,4 +57,4 @@ CometNativeColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.item -Comet accelerated 58 out of 61 eligible operators (95%). Final plan contains 1 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 54 out of 57 eligible operators (94%). Final plan contains 1 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q23a.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q23a.native_datafusion/extended.txt index 3e46e4a329..6dd0348345 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q23a.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q23a.native_datafusion/extended.txt @@ -94,11 +94,7 @@ CometNativeColumnarToRow : : +- CometProject : : +- CometBroadcastHashJoin : : :- CometNativeScan parquet spark_catalog.default.web_sales - : : : +- CometSubqueryBroadcast - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- ReusedSubquery : : +- CometBroadcastExchange : : +- CometProject : : +- CometFilter @@ -143,4 +139,4 @@ CometNativeColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.date_dim -Comet accelerated 135 out of 142 eligible operators (95%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 131 out of 138 eligible operators (94%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q23b.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q23b.native_datafusion/extended.txt index 79baaf1aa9..f2228c277c 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q23b.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q23b.native_datafusion/extended.txt @@ -124,11 +124,7 @@ CometNativeColumnarToRow : : : +- CometBroadcastHashJoin : : : :- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : : +- CometSubqueryBroadcast - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- ReusedSubquery : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter @@ -195,4 +191,4 @@ CometNativeColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.date_dim -Comet accelerated 185 out of 194 eligible operators (95%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 181 out of 190 eligible operators (95%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q25.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q25.native_datafusion/extended.txt index 2650cd8f1a..a3f43554cf 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q25.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q25.native_datafusion/extended.txt @@ -35,11 +35,7 @@ CometNativeColumnarToRow : : : : : +- CometBroadcastExchange : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : : +- CometSubqueryBroadcast - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- ReusedSubquery : : : : +- CometBroadcastExchange : : : : +- CometProject : : : : +- CometFilter @@ -61,4 +57,4 @@ CometNativeColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.item -Comet accelerated 58 out of 61 eligible operators (95%). Final plan contains 1 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 54 out of 57 eligible operators (94%). Final plan contains 1 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q30.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q30.native_datafusion/extended.txt index 2a22518df4..3f88234deb 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q30.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q30.native_datafusion/extended.txt @@ -43,11 +43,7 @@ CometNativeColumnarToRow : : : +- CometBroadcastHashJoin : : : :- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.web_returns - : : : : +- CometSubqueryBroadcast - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- ReusedSubquery : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter @@ -65,4 +61,4 @@ CometNativeColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.customer_address -Comet accelerated 63 out of 65 eligible operators (96%). Final plan contains 1 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 59 out of 61 eligible operators (96%). Final plan contains 1 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q31.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q31.native_datafusion/extended.txt index bf4fb2aa77..188e553f1a 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q31.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q31.native_datafusion/extended.txt @@ -78,10 +78,7 @@ CometNativeColumnarToRow : : : +- CometBroadcastHashJoin : : : :- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : : +- CometSubqueryBroadcast - : : : : +- CometBroadcastExchange - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- ReusedSubquery : : : +- CometBroadcastExchange : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -98,10 +95,7 @@ CometNativeColumnarToRow : : +- CometBroadcastHashJoin : : :- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : +- CometSubqueryBroadcast - : : : +- CometBroadcastExchange - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- ReusedSubquery : : +- CometBroadcastExchange : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -118,10 +112,7 @@ CometNativeColumnarToRow : +- CometBroadcastHashJoin : :- CometFilter : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : +- CometSubqueryBroadcast - : : +- CometBroadcastExchange - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- ReusedSubquery : +- CometBroadcastExchange : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -129,4 +120,4 @@ CometNativeColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.customer_address -Comet accelerated 123 out of 129 eligible operators (95%). Final plan contains 1 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 114 out of 120 eligible operators (95%). Final plan contains 1 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q32.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q32.native_datafusion/extended.txt index db07e648bc..a536c4b38f 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q32.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q32.native_datafusion/extended.txt @@ -28,11 +28,7 @@ CometNativeColumnarToRow : +- CometBroadcastHashJoin : :- CometFilter : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : +- CometSubqueryBroadcast - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- ReusedSubquery : +- CometBroadcastExchange : +- CometProject : +- CometFilter @@ -42,4 +38,4 @@ CometNativeColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.date_dim -Comet accelerated 40 out of 42 eligible operators (95%). Final plan contains 1 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 36 out of 38 eligible operators (94%). Final plan contains 1 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q33.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q33.native_datafusion/extended.txt index d64ba66952..9f17b73589 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q33.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q33.native_datafusion/extended.txt @@ -47,11 +47,7 @@ CometNativeColumnarToRow : : : +- CometBroadcastHashJoin : : : :- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : +- CometSubqueryBroadcast - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- ReusedSubquery : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter @@ -79,11 +75,7 @@ CometNativeColumnarToRow : : +- CometBroadcastHashJoin : : :- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : +- CometSubqueryBroadcast - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- ReusedSubquery : : +- CometBroadcastExchange : : +- CometProject : : +- CometFilter @@ -101,4 +93,4 @@ CometNativeColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.item -Comet accelerated 98 out of 101 eligible operators (97%). Final plan contains 1 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 90 out of 93 eligible operators (96%). Final plan contains 1 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q35.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q35.native_datafusion/extended.txt index 2e79f6f464..b7c8209556 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q35.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q35.native_datafusion/extended.txt @@ -33,11 +33,7 @@ TakeOrderedAndProject : : : +- CometProject : : : +- CometBroadcastHashJoin : : : :- CometNativeScan parquet spark_catalog.default.web_sales - : : : : +- CometSubqueryBroadcast - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- ReusedSubquery : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter @@ -47,11 +43,7 @@ TakeOrderedAndProject : : +- CometProject : : +- CometBroadcastHashJoin : : :- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : +- CometSubqueryBroadcast - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- ReusedSubquery : : +- CometBroadcastExchange : : +- CometProject : : +- CometFilter @@ -67,4 +59,4 @@ TakeOrderedAndProject +- CometFilter +- CometNativeScan parquet spark_catalog.default.customer_demographics -Comet accelerated 44 out of 62 eligible operators (70%). Final plan contains 6 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 36 out of 54 eligible operators (66%). Final plan contains 6 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q38.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q38.native_datafusion/extended.txt index c0314d8fb0..c965be9cc3 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q38.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q38.native_datafusion/extended.txt @@ -37,11 +37,7 @@ CometNativeColumnarToRow : : +- CometBroadcastHashJoin : : :- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : +- CometSubqueryBroadcast - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- ReusedSubquery : : +- CometBroadcastExchange : : +- CometProject : : +- CometFilter @@ -60,11 +56,7 @@ CometNativeColumnarToRow : +- CometBroadcastHashJoin : :- CometFilter : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : +- CometSubqueryBroadcast - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- ReusedSubquery : +- CometBroadcastExchange : +- CometProject : +- CometFilter @@ -74,4 +66,4 @@ CometNativeColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.customer -Comet accelerated 71 out of 74 eligible operators (95%). Final plan contains 1 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 63 out of 66 eligible operators (95%). Final plan contains 1 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q4.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q4.native_datafusion/extended.txt index bf61cfd81e..ff66588e1a 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q4.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q4.native_datafusion/extended.txt @@ -66,10 +66,7 @@ CometNativeColumnarToRow : : : : +- CometBroadcastExchange : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : +- CometSubqueryBroadcast - : : : : +- CometBroadcastExchange - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- ReusedSubquery : : : +- CometBroadcastExchange : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -87,10 +84,7 @@ CometNativeColumnarToRow : : : +- CometBroadcastExchange : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : +- CometSubqueryBroadcast - : : : +- CometBroadcastExchange - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- ReusedSubquery : : +- CometBroadcastExchange : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -109,10 +103,7 @@ CometNativeColumnarToRow : : +- CometBroadcastExchange : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : +- CometSubqueryBroadcast - : : +- CometBroadcastExchange - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- ReusedSubquery : +- CometBroadcastExchange : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -130,12 +121,9 @@ CometNativeColumnarToRow : +- CometBroadcastExchange : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.web_sales - : +- CometSubqueryBroadcast - : +- CometBroadcastExchange - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- ReusedSubquery +- CometBroadcastExchange +- CometFilter +- CometNativeScan parquet spark_catalog.default.date_dim -Comet accelerated 132 out of 138 eligible operators (95%). Final plan contains 1 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 120 out of 126 eligible operators (95%). Final plan contains 1 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q49.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q49.native_datafusion/extended.txt index f2ae44fb86..cd6c39e4f3 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q49.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q49.native_datafusion/extended.txt @@ -54,11 +54,7 @@ CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : +- CometSubqueryBroadcast - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- ReusedSubquery : : +- CometProject : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.catalog_returns @@ -85,11 +81,7 @@ CometNativeColumnarToRow : : +- CometProject : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : +- CometSubqueryBroadcast - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- ReusedSubquery : +- CometProject : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.store_returns @@ -98,4 +90,4 @@ CometNativeColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.date_dim -Comet accelerated 75 out of 95 eligible operators (78%). Final plan contains 4 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 67 out of 87 eligible operators (77%). Final plan contains 4 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q5.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q5.native_datafusion/extended.txt index adef074558..7fcc516b17 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q5.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q5.native_datafusion/extended.txt @@ -24,11 +24,7 @@ CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.store_returns - : : : +- CometSubqueryBroadcast - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- ReusedSubquery : : +- CometBroadcastExchange : : +- CometProject : : +- CometFilter @@ -48,19 +44,11 @@ CometNativeColumnarToRow : : : :- CometProject : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : +- CometSubqueryBroadcast - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- ReusedSubquery : : : +- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.catalog_returns - : : : +- CometSubqueryBroadcast - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- ReusedSubquery : : +- CometBroadcastExchange : : +- CometProject : : +- CometFilter @@ -80,20 +68,12 @@ CometNativeColumnarToRow : : :- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : +- CometSubqueryBroadcast - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- ReusedSubquery : : +- CometProject : : +- CometBroadcastHashJoin : : :- CometBroadcastExchange : : : +- CometNativeScan parquet spark_catalog.default.web_returns - : : : +- CometSubqueryBroadcast - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- ReusedSubquery : : +- CometProject : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.web_sales @@ -106,4 +86,4 @@ CometNativeColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.web_site -Comet accelerated 100 out of 106 eligible operators (94%). Final plan contains 1 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 80 out of 86 eligible operators (93%). Final plan contains 1 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q51.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q51.native_datafusion/extended.txt index 173d1f064f..f67067735c 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q51.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q51.native_datafusion/extended.txt @@ -43,14 +43,10 @@ TakeOrderedAndProject +- CometBroadcastHashJoin :- CometFilter : +- CometNativeScan parquet spark_catalog.default.store_sales - : +- CometSubqueryBroadcast - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- ReusedSubquery +- CometBroadcastExchange +- CometProject +- CometFilter +- CometNativeScan parquet spark_catalog.default.date_dim -Comet accelerated 42 out of 51 eligible operators (82%). Final plan contains 3 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 38 out of 47 eligible operators (80%). Final plan contains 3 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q54.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q54.native_datafusion/extended.txt index 9e1691c2fc..af2b39b224 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q54.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q54.native_datafusion/extended.txt @@ -35,11 +35,7 @@ CometNativeColumnarToRow : : : : : : : +- CometProject : : : : : : : +- CometFilter : : : : : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : : : : : +- CometSubqueryBroadcast - : : : : : : : +- CometBroadcastExchange - : : : : : : : +- CometProject - : : : : : : : +- CometFilter - : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : : +- ReusedSubquery : : : : : : +- CometBroadcastExchange : : : : : : +- CometProject : : : : : : +- CometFilter @@ -104,4 +100,4 @@ CometNativeColumnarToRow : +- CometNativeScan parquet spark_catalog.default.date_dim +- CometNativeScan parquet spark_catalog.default.date_dim -Comet accelerated 93 out of 100 eligible operators (93%). Final plan contains 5 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 89 out of 96 eligible operators (92%). Final plan contains 5 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q56.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q56.native_datafusion/extended.txt index 0744bc307d..e8b3e8e97f 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q56.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q56.native_datafusion/extended.txt @@ -48,11 +48,7 @@ CometNativeColumnarToRow : : : +- CometBroadcastHashJoin : : : :- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : +- CometSubqueryBroadcast - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- ReusedSubquery : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter @@ -81,11 +77,7 @@ CometNativeColumnarToRow : : +- CometBroadcastHashJoin : : :- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : +- CometSubqueryBroadcast - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- ReusedSubquery : : +- CometBroadcastExchange : : +- CometProject : : +- CometFilter @@ -104,4 +96,4 @@ CometNativeColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.item -Comet accelerated 101 out of 104 eligible operators (97%). Final plan contains 1 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 93 out of 96 eligible operators (96%). Final plan contains 1 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q58.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q58.native_datafusion/extended.txt index 2e5b58d22e..7f1bcd05ac 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q58.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q58.native_datafusion/extended.txt @@ -58,21 +58,7 @@ CometNativeColumnarToRow : : +- CometBroadcastHashJoin : : :- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : +- CometSubqueryBroadcast - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : : +- Subquery - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- ReusedSubquery : : +- CometBroadcastExchange : : +- CometProject : : +- CometFilter @@ -102,21 +88,7 @@ CometNativeColumnarToRow : +- CometBroadcastHashJoin : :- CometFilter : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : +- CometSubqueryBroadcast - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : : +- Subquery - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- ReusedSubquery : +- CometBroadcastExchange : +- CometProject : +- CometFilter @@ -136,4 +108,4 @@ CometNativeColumnarToRow : +- CometNativeScan parquet spark_catalog.default.date_dim +- CometNativeScan parquet spark_catalog.default.date_dim -Comet accelerated 121 out of 130 eligible operators (93%). Final plan contains 7 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 97 out of 104 eligible operators (93%). Final plan contains 5 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q60.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q60.native_datafusion/extended.txt index 0744bc307d..e8b3e8e97f 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q60.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q60.native_datafusion/extended.txt @@ -48,11 +48,7 @@ CometNativeColumnarToRow : : : +- CometBroadcastHashJoin : : : :- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : +- CometSubqueryBroadcast - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- ReusedSubquery : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter @@ -81,11 +77,7 @@ CometNativeColumnarToRow : : +- CometBroadcastHashJoin : : :- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : +- CometSubqueryBroadcast - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- ReusedSubquery : : +- CometBroadcastExchange : : +- CometProject : : +- CometFilter @@ -104,4 +96,4 @@ CometNativeColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.item -Comet accelerated 101 out of 104 eligible operators (97%). Final plan contains 1 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 93 out of 96 eligible operators (96%). Final plan contains 1 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q61.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q61.native_datafusion/extended.txt index 8d91cbe7a2..39f985df28 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q61.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q61.native_datafusion/extended.txt @@ -63,11 +63,7 @@ Project : : : : +- CometBroadcastHashJoin : : : : :- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : : : +- CometSubqueryBroadcast - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- ReusedSubquery : : : : +- CometBroadcastExchange : : : : +- CometProject : : : : +- CometFilter @@ -88,4 +84,4 @@ Project +- CometFilter +- CometNativeScan parquet spark_catalog.default.item -Comet accelerated 82 out of 87 eligible operators (94%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 78 out of 83 eligible operators (93%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q65.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q65.native_datafusion/extended.txt index 21d60497a1..b3937f6c97 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q65.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q65.native_datafusion/extended.txt @@ -42,14 +42,10 @@ CometNativeColumnarToRow +- CometBroadcastHashJoin :- CometFilter : +- CometNativeScan parquet spark_catalog.default.store_sales - : +- CometSubqueryBroadcast - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- ReusedSubquery +- CometBroadcastExchange +- CometProject +- CometFilter +- CometNativeScan parquet spark_catalog.default.date_dim -Comet accelerated 50 out of 52 eligible operators (96%). Final plan contains 1 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 46 out of 48 eligible operators (95%). Final plan contains 1 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q66.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q66.native_datafusion/extended.txt index c07ecfe46a..3b8d1937f2 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q66.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q66.native_datafusion/extended.txt @@ -49,10 +49,7 @@ CometNativeColumnarToRow : : : +- CometBroadcastHashJoin : : : :- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : +- CometSubqueryBroadcast - : : : : +- CometBroadcastExchange - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- ReusedSubquery : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter @@ -69,4 +66,4 @@ CometNativeColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.ship_mode -Comet accelerated 67 out of 69 eligible operators (97%). Final plan contains 1 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 64 out of 66 eligible operators (96%). Final plan contains 1 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q69.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q69.native_datafusion/extended.txt index 6d398580fb..cb518a5c47 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q69.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q69.native_datafusion/extended.txt @@ -32,11 +32,7 @@ TakeOrderedAndProject : : : +- CometProject : : : +- CometBroadcastHashJoin : : : :- CometNativeScan parquet spark_catalog.default.web_sales - : : : : +- CometSubqueryBroadcast - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- ReusedSubquery : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter @@ -46,11 +42,7 @@ TakeOrderedAndProject : : +- CometProject : : +- CometBroadcastHashJoin : : :- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : +- CometSubqueryBroadcast - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- ReusedSubquery : : +- CometBroadcastExchange : : +- CometProject : : +- CometFilter @@ -66,4 +58,4 @@ TakeOrderedAndProject +- CometFilter +- CometNativeScan parquet spark_catalog.default.customer_demographics -Comet accelerated 44 out of 61 eligible operators (72%). Final plan contains 6 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 36 out of 53 eligible operators (67%). Final plan contains 6 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q70.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q70.native_datafusion/extended.txt index d1ff1b903e..173923495f 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q70.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q70.native_datafusion/extended.txt @@ -47,11 +47,7 @@ TakeOrderedAndProject : +- CometBroadcastHashJoin : :- CometFilter : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : +- CometSubqueryBroadcast - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- ReusedSubquery : +- CometBroadcastExchange : +- CometProject : +- CometFilter @@ -61,4 +57,4 @@ TakeOrderedAndProject +- CometFilter +- CometNativeScan parquet spark_catalog.default.date_dim -Comet accelerated 39 out of 57 eligible operators (68%). Final plan contains 5 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 35 out of 53 eligible operators (66%). Final plan contains 5 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q71.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q71.native_datafusion/extended.txt index 834098d4ee..2c056d01d7 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q71.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q71.native_datafusion/extended.txt @@ -30,11 +30,7 @@ CometNativeColumnarToRow : : +- CometBroadcastHashJoin : : :- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : +- CometSubqueryBroadcast - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- ReusedSubquery : : +- CometBroadcastExchange : : +- CometProject : : +- CometFilter @@ -43,11 +39,7 @@ CometNativeColumnarToRow : +- CometBroadcastHashJoin : :- CometFilter : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : +- CometSubqueryBroadcast - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- ReusedSubquery : +- CometBroadcastExchange : +- CometProject : +- CometFilter @@ -57,4 +49,4 @@ CometNativeColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.time_dim -Comet accelerated 54 out of 57 eligible operators (94%). Final plan contains 1 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 46 out of 49 eligible operators (93%). Final plan contains 1 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q74.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q74.native_datafusion/extended.txt index 7768138645..cfe634568f 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q74.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q74.native_datafusion/extended.txt @@ -62,10 +62,7 @@ CometNativeColumnarToRow : : +- CometBroadcastExchange : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : +- CometSubqueryBroadcast - : : +- CometBroadcastExchange - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- ReusedSubquery : +- CometBroadcastExchange : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -83,12 +80,9 @@ CometNativeColumnarToRow : +- CometBroadcastExchange : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.web_sales - : +- CometSubqueryBroadcast - : +- CometBroadcastExchange - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- ReusedSubquery +- CometBroadcastExchange +- CometFilter +- CometNativeScan parquet spark_catalog.default.date_dim -Comet accelerated 87 out of 91 eligible operators (95%). Final plan contains 1 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 81 out of 85 eligible operators (95%). Final plan contains 1 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q75.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q75.native_datafusion/extended.txt index ee7c0af216..bccd23b141 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q75.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q75.native_datafusion/extended.txt @@ -48,10 +48,7 @@ CometNativeColumnarToRow : : : : +- CometBroadcastHashJoin : : : : :- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : : : +- CometSubqueryBroadcast - : : : : : +- CometBroadcastExchange - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- ReusedSubquery : : : : +- CometBroadcastExchange : : : : +- CometProject : : : : +- CometFilter @@ -74,10 +71,7 @@ CometNativeColumnarToRow : : : +- CometBroadcastHashJoin : : : :- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : : +- CometSubqueryBroadcast - : : : : +- CometBroadcastExchange - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- ReusedSubquery : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter @@ -136,10 +130,7 @@ CometNativeColumnarToRow : : : +- CometBroadcastHashJoin : : : :- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : : +- CometSubqueryBroadcast - : : : : +- CometBroadcastExchange - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- ReusedSubquery : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter @@ -162,10 +153,7 @@ CometNativeColumnarToRow : : +- CometBroadcastHashJoin : : :- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : +- CometSubqueryBroadcast - : : : +- CometBroadcastExchange - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- ReusedSubquery : : +- CometBroadcastExchange : : +- CometProject : : +- CometFilter @@ -179,4 +167,4 @@ CometNativeColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.web_returns -Comet accelerated 173 out of 179 eligible operators (96%). Final plan contains 1 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 161 out of 167 eligible operators (96%). Final plan contains 1 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q77.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q77.native_datafusion/extended.txt index 8805a2e6c2..38ee51e753 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q77.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q77.native_datafusion/extended.txt @@ -39,11 +39,7 @@ TakeOrderedAndProject : : +- CometBroadcastHashJoin : : :- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.store_returns - : : : +- CometSubqueryBroadcast - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- ReusedSubquery : : +- CometBroadcastExchange : : +- CometProject : : +- CometFilter @@ -61,11 +57,7 @@ TakeOrderedAndProject : : +- CometProject : : +- CometBroadcastHashJoin : : :- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : +- CometSubqueryBroadcast - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- ReusedSubquery : : +- CometBroadcastExchange : : +- CometProject : : +- CometFilter @@ -77,11 +69,7 @@ TakeOrderedAndProject : +- CometProject : +- CometBroadcastHashJoin : :- CometNativeScan parquet spark_catalog.default.catalog_returns - : : +- CometSubqueryBroadcast - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- ReusedSubquery : +- CometBroadcastExchange : +- CometProject : +- CometFilter @@ -98,11 +86,7 @@ TakeOrderedAndProject : : +- CometBroadcastHashJoin : : :- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : +- CometSubqueryBroadcast - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- ReusedSubquery : : +- CometBroadcastExchange : : +- CometProject : : +- CometFilter @@ -120,11 +104,7 @@ TakeOrderedAndProject : +- CometBroadcastHashJoin : :- CometFilter : : +- CometNativeScan parquet spark_catalog.default.web_returns - : : +- CometSubqueryBroadcast - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- ReusedSubquery : +- CometBroadcastExchange : +- CometProject : +- CometFilter @@ -133,4 +113,4 @@ TakeOrderedAndProject +- CometFilter +- CometNativeScan parquet spark_catalog.default.web_page -Comet accelerated 115 out of 129 eligible operators (89%). Final plan contains 5 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 95 out of 109 eligible operators (87%). Final plan contains 5 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q78.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q78.native_datafusion/extended.txt index 8426b9653c..d29dbc13e5 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q78.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q78.native_datafusion/extended.txt @@ -43,10 +43,7 @@ TakeOrderedAndProject : : : +- CometExchange : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : +- CometSubqueryBroadcast - : : : +- CometBroadcastExchange - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- ReusedSubquery : : +- CometSort : : +- CometExchange : : +- CometProject @@ -69,10 +66,7 @@ TakeOrderedAndProject : : +- CometExchange : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : +- CometSubqueryBroadcast - : : +- CometBroadcastExchange - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- ReusedSubquery : +- CometSort : +- CometExchange : +- CometProject @@ -82,4 +76,4 @@ TakeOrderedAndProject +- CometFilter +- CometNativeScan parquet spark_catalog.default.date_dim -Comet accelerated 77 out of 82 eligible operators (93%). Final plan contains 1 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 71 out of 76 eligible operators (93%). Final plan contains 1 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q80.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q80.native_datafusion/extended.txt index 27e32a1602..9f67d9b8b5 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q80.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q80.native_datafusion/extended.txt @@ -65,11 +65,7 @@ CometNativeColumnarToRow : : : : : : +- CometExchange : : : : : : +- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : : : +- CometSubqueryBroadcast - : : : : : : +- CometBroadcastExchange - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : +- ReusedSubquery : : : : : +- CometSort : : : : : +- CometExchange : : : : : +- CometProject @@ -108,11 +104,7 @@ CometNativeColumnarToRow : : : : : +- CometExchange : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : : : +- CometSubqueryBroadcast - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- ReusedSubquery : : : : +- CometSort : : : : +- CometExchange : : : : +- CometProject @@ -135,4 +127,4 @@ CometNativeColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.promotion -Comet accelerated 132 out of 135 eligible operators (97%). Final plan contains 1 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 124 out of 127 eligible operators (97%). Final plan contains 1 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q81.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q81.native_datafusion/extended.txt index 6d9aa9697d..f10339641b 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q81.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q81.native_datafusion/extended.txt @@ -43,11 +43,7 @@ CometNativeColumnarToRow : : : +- CometBroadcastHashJoin : : : :- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.catalog_returns - : : : : +- CometSubqueryBroadcast - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- ReusedSubquery : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter @@ -65,4 +61,4 @@ CometNativeColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.customer_address -Comet accelerated 63 out of 65 eligible operators (96%). Final plan contains 1 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 59 out of 61 eligible operators (96%). Final plan contains 1 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q83.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q83.native_datafusion/extended.txt index e4d1992492..32adaba671 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q83.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q83.native_datafusion/extended.txt @@ -54,20 +54,7 @@ CometNativeColumnarToRow : : +- CometBroadcastHashJoin : : :- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.catalog_returns - : : : +- CometSubqueryBroadcast - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- ReusedSubquery : : +- CometBroadcastExchange : : +- CometProject : : +- CometFilter @@ -95,20 +82,7 @@ CometNativeColumnarToRow : +- CometBroadcastHashJoin : :- CometFilter : : +- CometNativeScan parquet spark_catalog.default.web_returns - : : +- CometSubqueryBroadcast - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometNativeScan parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- ReusedSubquery : +- CometBroadcastExchange : +- CometProject : +- CometFilter @@ -127,4 +101,4 @@ CometNativeColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.date_dim -Comet accelerated 124 out of 127 eligible operators (97%). Final plan contains 1 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 98 out of 101 eligible operators (97%). Final plan contains 1 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q87.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q87.native_datafusion/extended.txt index 5c80165c1f..7680e50295 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q87.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q87.native_datafusion/extended.txt @@ -39,11 +39,7 @@ HashAggregate : : +- CometBroadcastHashJoin : : :- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : +- CometSubqueryBroadcast - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- ReusedSubquery : : +- CometBroadcastExchange : : +- CometProject : : +- CometFilter @@ -63,11 +59,7 @@ HashAggregate : +- CometBroadcastHashJoin : :- CometFilter : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : +- CometSubqueryBroadcast - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- ReusedSubquery : +- CometBroadcastExchange : +- CometProject : +- CometFilter @@ -77,4 +69,4 @@ HashAggregate +- CometFilter +- CometNativeScan parquet spark_catalog.default.customer -Comet accelerated 64 out of 74 eligible operators (86%). Final plan contains 4 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 56 out of 66 eligible operators (84%). Final plan contains 4 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q92.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q92.native_datafusion/extended.txt index 40926fafac..bcf7db282b 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q92.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q92.native_datafusion/extended.txt @@ -28,11 +28,7 @@ CometNativeColumnarToRow : +- CometBroadcastHashJoin : :- CometFilter : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : +- CometSubqueryBroadcast - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- ReusedSubquery : +- CometBroadcastExchange : +- CometProject : +- CometFilter @@ -42,4 +38,4 @@ CometNativeColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.date_dim -Comet accelerated 40 out of 42 eligible operators (95%). Final plan contains 1 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 36 out of 38 eligible operators (94%). Final plan contains 1 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q97.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q97.native_datafusion/extended.txt index 74163dff2b..a56bd059fa 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q97.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q97.native_datafusion/extended.txt @@ -27,14 +27,10 @@ CometNativeColumnarToRow +- CometProject +- CometBroadcastHashJoin :- CometNativeScan parquet spark_catalog.default.catalog_sales - : +- CometSubqueryBroadcast - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- ReusedSubquery +- CometBroadcastExchange +- CometProject +- CometFilter +- CometNativeScan parquet spark_catalog.default.date_dim -Comet accelerated 35 out of 37 eligible operators (94%). Final plan contains 1 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 31 out of 33 eligible operators (93%). Final plan contains 1 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q1.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q1.native_datafusion/extended.txt index 5875fa3a5e..f49109dafb 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q1.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q1.native_datafusion/extended.txt @@ -35,11 +35,7 @@ CometNativeColumnarToRow : : +- CometBroadcastHashJoin : : :- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.store_returns - : : : +- CometSubqueryBroadcast - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- ReusedSubquery : : +- CometBroadcastExchange : : +- CometProject : : +- CometFilter @@ -53,4 +49,4 @@ CometNativeColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.customer -Comet accelerated 51 out of 53 eligible operators (96%). Final plan contains 1 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 47 out of 49 eligible operators (95%). Final plan contains 1 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q10.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q10.native_datafusion/extended.txt index 2e79f6f464..b7c8209556 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q10.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q10.native_datafusion/extended.txt @@ -33,11 +33,7 @@ TakeOrderedAndProject : : : +- CometProject : : : +- CometBroadcastHashJoin : : : :- CometNativeScan parquet spark_catalog.default.web_sales - : : : : +- CometSubqueryBroadcast - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- ReusedSubquery : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter @@ -47,11 +43,7 @@ TakeOrderedAndProject : : +- CometProject : : +- CometBroadcastHashJoin : : :- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : +- CometSubqueryBroadcast - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- ReusedSubquery : : +- CometBroadcastExchange : : +- CometProject : : +- CometFilter @@ -67,4 +59,4 @@ TakeOrderedAndProject +- CometFilter +- CometNativeScan parquet spark_catalog.default.customer_demographics -Comet accelerated 44 out of 62 eligible operators (70%). Final plan contains 6 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 36 out of 54 eligible operators (66%). Final plan contains 6 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q11.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q11.native_datafusion/extended.txt index edda694b73..7f06caa2e1 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q11.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q11.native_datafusion/extended.txt @@ -63,10 +63,7 @@ CometNativeColumnarToRow : : +- CometBroadcastExchange : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : +- CometSubqueryBroadcast - : : +- CometBroadcastExchange - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- ReusedSubquery : +- CometBroadcastExchange : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -84,12 +81,9 @@ CometNativeColumnarToRow : +- CometBroadcastExchange : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.web_sales - : +- CometSubqueryBroadcast - : +- CometBroadcastExchange - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- ReusedSubquery +- CometBroadcastExchange +- CometFilter +- CometNativeScan parquet spark_catalog.default.date_dim -Comet accelerated 88 out of 92 eligible operators (95%). Final plan contains 1 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 82 out of 86 eligible operators (95%). Final plan contains 1 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q14a.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q14a.native_datafusion/extended.txt index d2a44e9c52..1ae3f23b40 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q14a.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q14a.native_datafusion/extended.txt @@ -16,11 +16,7 @@ CometNativeColumnarToRow : : :- CometProject : : : +- CometBroadcastHashJoin : : : :- CometNativeScan parquet spark_catalog.default.store_sales - : : : : +- CometSubqueryBroadcast - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- ReusedSubquery : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter @@ -28,11 +24,7 @@ CometNativeColumnarToRow : : :- CometProject : : : +- CometBroadcastHashJoin : : : :- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : +- CometSubqueryBroadcast - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- ReusedSubquery : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter @@ -40,11 +32,7 @@ CometNativeColumnarToRow : : +- CometProject : : +- CometBroadcastHashJoin : : :- CometNativeScan parquet spark_catalog.default.web_sales - : : : +- CometSubqueryBroadcast - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- ReusedSubquery : : +- CometBroadcastExchange : : +- CometProject : : +- CometFilter @@ -96,11 +84,7 @@ CometNativeColumnarToRow : : : : : : +- CometBroadcastHashJoin : : : : : : :- CometFilter : : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : : : : +- CometSubqueryBroadcast - : : : : : : : +- CometBroadcastExchange - : : : : : : : +- CometProject - : : : : : : : +- CometFilter - : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : : +- ReusedSubquery : : : : : : +- CometBroadcastExchange : : : : : : +- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.item @@ -119,11 +103,7 @@ CometNativeColumnarToRow : : : : +- CometBroadcastHashJoin : : : : :- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : : : +- CometSubqueryBroadcast - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- ReusedSubquery : : : : +- CometBroadcastExchange : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.item @@ -167,11 +147,7 @@ CometNativeColumnarToRow : : : : : +- CometBroadcastHashJoin : : : : : :- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : : : +- CometSubqueryBroadcast - : : : : : : +- CometBroadcastExchange - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : +- ReusedSubquery : : : : : +- CometBroadcastExchange : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.item @@ -190,11 +166,7 @@ CometNativeColumnarToRow : : : +- CometBroadcastHashJoin : : : :- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : : +- CometSubqueryBroadcast - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- ReusedSubquery : : : +- CometBroadcastExchange : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.item @@ -219,11 +191,7 @@ CometNativeColumnarToRow : : :- CometBroadcastHashJoin : : : :- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : +- CometSubqueryBroadcast - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- ReusedSubquery : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometBroadcastHashJoin @@ -256,11 +224,7 @@ CometNativeColumnarToRow : : : : : : +- CometBroadcastHashJoin : : : : : : :- CometFilter : : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : : : : +- CometSubqueryBroadcast - : : : : : : : +- CometBroadcastExchange - : : : : : : : +- CometProject - : : : : : : : +- CometFilter - : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : : +- ReusedSubquery : : : : : : +- CometBroadcastExchange : : : : : : +- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.item @@ -279,11 +243,7 @@ CometNativeColumnarToRow : : : : +- CometBroadcastHashJoin : : : : :- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : : : +- CometSubqueryBroadcast - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- ReusedSubquery : : : : +- CometBroadcastExchange : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.item @@ -327,11 +287,7 @@ CometNativeColumnarToRow : : : : : +- CometBroadcastHashJoin : : : : : :- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : : : +- CometSubqueryBroadcast - : : : : : : +- CometBroadcastExchange - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : +- ReusedSubquery : : : : : +- CometBroadcastExchange : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.item @@ -350,11 +306,7 @@ CometNativeColumnarToRow : : : +- CometBroadcastHashJoin : : : :- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : : +- CometSubqueryBroadcast - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- ReusedSubquery : : : +- CometBroadcastExchange : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.item @@ -379,11 +331,7 @@ CometNativeColumnarToRow : :- CometBroadcastHashJoin : : :- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : +- CometSubqueryBroadcast - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- ReusedSubquery : : +- CometBroadcastExchange : : +- CometProject : : +- CometBroadcastHashJoin @@ -416,11 +364,7 @@ CometNativeColumnarToRow : : : : : +- CometBroadcastHashJoin : : : : : :- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : : : +- CometSubqueryBroadcast - : : : : : : +- CometBroadcastExchange - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : +- ReusedSubquery : : : : : +- CometBroadcastExchange : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.item @@ -439,11 +383,7 @@ CometNativeColumnarToRow : : : +- CometBroadcastHashJoin : : : :- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : : +- CometSubqueryBroadcast - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- ReusedSubquery : : : +- CometBroadcastExchange : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.item @@ -487,11 +427,7 @@ CometNativeColumnarToRow : : : : +- CometBroadcastHashJoin : : : : :- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : : +- CometSubqueryBroadcast - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- ReusedSubquery : : : : +- CometBroadcastExchange : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.item @@ -510,11 +446,7 @@ CometNativeColumnarToRow : : +- CometBroadcastHashJoin : : :- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : +- CometSubqueryBroadcast - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- ReusedSubquery : : +- CometBroadcastExchange : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.item @@ -527,4 +459,4 @@ CometNativeColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.date_dim -Comet accelerated 499 out of 526 eligible operators (94%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 431 out of 458 eligible operators (94%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q14b.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q14b.native_datafusion/extended.txt index 341ae1fb04..85cda8b553 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q14b.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q14b.native_datafusion/extended.txt @@ -11,11 +11,7 @@ CometNativeColumnarToRow : : :- CometProject : : : +- CometBroadcastHashJoin : : : :- CometNativeScan parquet spark_catalog.default.store_sales - : : : : +- CometSubqueryBroadcast - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- ReusedSubquery : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter @@ -23,11 +19,7 @@ CometNativeColumnarToRow : : :- CometProject : : : +- CometBroadcastHashJoin : : : :- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : +- CometSubqueryBroadcast - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- ReusedSubquery : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter @@ -35,11 +27,7 @@ CometNativeColumnarToRow : : +- CometProject : : +- CometBroadcastHashJoin : : :- CometNativeScan parquet spark_catalog.default.web_sales - : : : +- CometSubqueryBroadcast - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- ReusedSubquery : : +- CometBroadcastExchange : : +- CometProject : : +- CometFilter @@ -58,11 +46,7 @@ CometNativeColumnarToRow : : : : +- CometBroadcastExchange : : : : +- CometProject : : : : +- CometFilter - : : : : : +- Subquery - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- ReusedSubquery : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : +- Subquery : : : : +- CometNativeColumnarToRow @@ -101,11 +85,7 @@ CometNativeColumnarToRow : : : : : : +- CometBroadcastHashJoin : : : : : : :- CometFilter : : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : : : : +- CometSubqueryBroadcast - : : : : : : : +- CometBroadcastExchange - : : : : : : : +- CometProject - : : : : : : : +- CometFilter - : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : : +- ReusedSubquery : : : : : : +- CometBroadcastExchange : : : : : : +- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.item @@ -124,11 +104,7 @@ CometNativeColumnarToRow : : : : +- CometBroadcastHashJoin : : : : :- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : : : +- CometSubqueryBroadcast - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- ReusedSubquery : : : : +- CometBroadcastExchange : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.item @@ -172,11 +148,7 @@ CometNativeColumnarToRow : : : : : +- CometBroadcastHashJoin : : : : : :- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : : : +- CometSubqueryBroadcast - : : : : : : +- CometBroadcastExchange - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : +- ReusedSubquery : : : : : +- CometBroadcastExchange : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.item @@ -195,11 +167,7 @@ CometNativeColumnarToRow : : : +- CometBroadcastHashJoin : : : :- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : : +- CometSubqueryBroadcast - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- ReusedSubquery : : : +- CometBroadcastExchange : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.item @@ -234,11 +202,7 @@ CometNativeColumnarToRow : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter - : : : : +- Subquery - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- ReusedSubquery : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- Subquery : : : +- CometNativeColumnarToRow @@ -277,11 +241,7 @@ CometNativeColumnarToRow : : : : : +- CometBroadcastHashJoin : : : : : :- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : : : +- CometSubqueryBroadcast - : : : : : : +- CometBroadcastExchange - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : +- ReusedSubquery : : : : : +- CometBroadcastExchange : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.item @@ -300,11 +260,7 @@ CometNativeColumnarToRow : : : +- CometBroadcastHashJoin : : : :- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : : +- CometSubqueryBroadcast - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- ReusedSubquery : : : +- CometBroadcastExchange : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.item @@ -348,11 +304,7 @@ CometNativeColumnarToRow : : : : +- CometBroadcastHashJoin : : : : :- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : : +- CometSubqueryBroadcast - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- ReusedSubquery : : : : +- CometBroadcastExchange : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.item @@ -371,11 +323,7 @@ CometNativeColumnarToRow : : +- CometBroadcastHashJoin : : :- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : +- CometSubqueryBroadcast - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- ReusedSubquery : : +- CometBroadcastExchange : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.item @@ -394,4 +342,4 @@ CometNativeColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.date_dim -Comet accelerated 360 out of 387 eligible operators (93%). Final plan contains 8 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 310 out of 337 eligible operators (91%). Final plan contains 6 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q17.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q17.native_datafusion/extended.txt index 2650cd8f1a..a3f43554cf 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q17.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q17.native_datafusion/extended.txt @@ -35,11 +35,7 @@ CometNativeColumnarToRow : : : : : +- CometBroadcastExchange : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : : +- CometSubqueryBroadcast - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- ReusedSubquery : : : : +- CometBroadcastExchange : : : : +- CometProject : : : : +- CometFilter @@ -61,4 +57,4 @@ CometNativeColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.item -Comet accelerated 58 out of 61 eligible operators (95%). Final plan contains 1 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 54 out of 57 eligible operators (94%). Final plan contains 1 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q23a.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q23a.native_datafusion/extended.txt index 3e46e4a329..6dd0348345 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q23a.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q23a.native_datafusion/extended.txt @@ -94,11 +94,7 @@ CometNativeColumnarToRow : : +- CometProject : : +- CometBroadcastHashJoin : : :- CometNativeScan parquet spark_catalog.default.web_sales - : : : +- CometSubqueryBroadcast - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- ReusedSubquery : : +- CometBroadcastExchange : : +- CometProject : : +- CometFilter @@ -143,4 +139,4 @@ CometNativeColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.date_dim -Comet accelerated 135 out of 142 eligible operators (95%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 131 out of 138 eligible operators (94%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q23b.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q23b.native_datafusion/extended.txt index 79baaf1aa9..f2228c277c 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q23b.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q23b.native_datafusion/extended.txt @@ -124,11 +124,7 @@ CometNativeColumnarToRow : : : +- CometBroadcastHashJoin : : : :- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : : +- CometSubqueryBroadcast - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- ReusedSubquery : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter @@ -195,4 +191,4 @@ CometNativeColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.date_dim -Comet accelerated 185 out of 194 eligible operators (95%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 181 out of 190 eligible operators (95%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q25.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q25.native_datafusion/extended.txt index 2650cd8f1a..a3f43554cf 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q25.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q25.native_datafusion/extended.txt @@ -35,11 +35,7 @@ CometNativeColumnarToRow : : : : : +- CometBroadcastExchange : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : : +- CometSubqueryBroadcast - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- ReusedSubquery : : : : +- CometBroadcastExchange : : : : +- CometProject : : : : +- CometFilter @@ -61,4 +57,4 @@ CometNativeColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.item -Comet accelerated 58 out of 61 eligible operators (95%). Final plan contains 1 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 54 out of 57 eligible operators (94%). Final plan contains 1 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q30.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q30.native_datafusion/extended.txt index 2a22518df4..3f88234deb 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q30.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q30.native_datafusion/extended.txt @@ -43,11 +43,7 @@ CometNativeColumnarToRow : : : +- CometBroadcastHashJoin : : : :- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.web_returns - : : : : +- CometSubqueryBroadcast - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- ReusedSubquery : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter @@ -65,4 +61,4 @@ CometNativeColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.customer_address -Comet accelerated 63 out of 65 eligible operators (96%). Final plan contains 1 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 59 out of 61 eligible operators (96%). Final plan contains 1 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q31.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q31.native_datafusion/extended.txt index bf4fb2aa77..188e553f1a 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q31.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q31.native_datafusion/extended.txt @@ -78,10 +78,7 @@ CometNativeColumnarToRow : : : +- CometBroadcastHashJoin : : : :- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : : +- CometSubqueryBroadcast - : : : : +- CometBroadcastExchange - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- ReusedSubquery : : : +- CometBroadcastExchange : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -98,10 +95,7 @@ CometNativeColumnarToRow : : +- CometBroadcastHashJoin : : :- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : +- CometSubqueryBroadcast - : : : +- CometBroadcastExchange - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- ReusedSubquery : : +- CometBroadcastExchange : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -118,10 +112,7 @@ CometNativeColumnarToRow : +- CometBroadcastHashJoin : :- CometFilter : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : +- CometSubqueryBroadcast - : : +- CometBroadcastExchange - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- ReusedSubquery : +- CometBroadcastExchange : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -129,4 +120,4 @@ CometNativeColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.customer_address -Comet accelerated 123 out of 129 eligible operators (95%). Final plan contains 1 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 114 out of 120 eligible operators (95%). Final plan contains 1 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q32.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q32.native_datafusion/extended.txt index db07e648bc..a536c4b38f 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q32.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q32.native_datafusion/extended.txt @@ -28,11 +28,7 @@ CometNativeColumnarToRow : +- CometBroadcastHashJoin : :- CometFilter : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : +- CometSubqueryBroadcast - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- ReusedSubquery : +- CometBroadcastExchange : +- CometProject : +- CometFilter @@ -42,4 +38,4 @@ CometNativeColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.date_dim -Comet accelerated 40 out of 42 eligible operators (95%). Final plan contains 1 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 36 out of 38 eligible operators (94%). Final plan contains 1 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q33.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q33.native_datafusion/extended.txt index d64ba66952..9f17b73589 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q33.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q33.native_datafusion/extended.txt @@ -47,11 +47,7 @@ CometNativeColumnarToRow : : : +- CometBroadcastHashJoin : : : :- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : +- CometSubqueryBroadcast - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- ReusedSubquery : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter @@ -79,11 +75,7 @@ CometNativeColumnarToRow : : +- CometBroadcastHashJoin : : :- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : +- CometSubqueryBroadcast - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- ReusedSubquery : : +- CometBroadcastExchange : : +- CometProject : : +- CometFilter @@ -101,4 +93,4 @@ CometNativeColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.item -Comet accelerated 98 out of 101 eligible operators (97%). Final plan contains 1 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 90 out of 93 eligible operators (96%). Final plan contains 1 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q35.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q35.native_datafusion/extended.txt index 2e79f6f464..b7c8209556 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q35.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q35.native_datafusion/extended.txt @@ -33,11 +33,7 @@ TakeOrderedAndProject : : : +- CometProject : : : +- CometBroadcastHashJoin : : : :- CometNativeScan parquet spark_catalog.default.web_sales - : : : : +- CometSubqueryBroadcast - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- ReusedSubquery : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter @@ -47,11 +43,7 @@ TakeOrderedAndProject : : +- CometProject : : +- CometBroadcastHashJoin : : :- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : +- CometSubqueryBroadcast - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- ReusedSubquery : : +- CometBroadcastExchange : : +- CometProject : : +- CometFilter @@ -67,4 +59,4 @@ TakeOrderedAndProject +- CometFilter +- CometNativeScan parquet spark_catalog.default.customer_demographics -Comet accelerated 44 out of 62 eligible operators (70%). Final plan contains 6 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 36 out of 54 eligible operators (66%). Final plan contains 6 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q38.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q38.native_datafusion/extended.txt index c0314d8fb0..c965be9cc3 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q38.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q38.native_datafusion/extended.txt @@ -37,11 +37,7 @@ CometNativeColumnarToRow : : +- CometBroadcastHashJoin : : :- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : +- CometSubqueryBroadcast - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- ReusedSubquery : : +- CometBroadcastExchange : : +- CometProject : : +- CometFilter @@ -60,11 +56,7 @@ CometNativeColumnarToRow : +- CometBroadcastHashJoin : :- CometFilter : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : +- CometSubqueryBroadcast - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- ReusedSubquery : +- CometBroadcastExchange : +- CometProject : +- CometFilter @@ -74,4 +66,4 @@ CometNativeColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.customer -Comet accelerated 71 out of 74 eligible operators (95%). Final plan contains 1 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 63 out of 66 eligible operators (95%). Final plan contains 1 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q4.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q4.native_datafusion/extended.txt index bf61cfd81e..ff66588e1a 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q4.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q4.native_datafusion/extended.txt @@ -66,10 +66,7 @@ CometNativeColumnarToRow : : : : +- CometBroadcastExchange : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : +- CometSubqueryBroadcast - : : : : +- CometBroadcastExchange - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- ReusedSubquery : : : +- CometBroadcastExchange : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -87,10 +84,7 @@ CometNativeColumnarToRow : : : +- CometBroadcastExchange : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : +- CometSubqueryBroadcast - : : : +- CometBroadcastExchange - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- ReusedSubquery : : +- CometBroadcastExchange : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -109,10 +103,7 @@ CometNativeColumnarToRow : : +- CometBroadcastExchange : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : +- CometSubqueryBroadcast - : : +- CometBroadcastExchange - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- ReusedSubquery : +- CometBroadcastExchange : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -130,12 +121,9 @@ CometNativeColumnarToRow : +- CometBroadcastExchange : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.web_sales - : +- CometSubqueryBroadcast - : +- CometBroadcastExchange - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- ReusedSubquery +- CometBroadcastExchange +- CometFilter +- CometNativeScan parquet spark_catalog.default.date_dim -Comet accelerated 132 out of 138 eligible operators (95%). Final plan contains 1 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 120 out of 126 eligible operators (95%). Final plan contains 1 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q49.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q49.native_datafusion/extended.txt index f2ae44fb86..cd6c39e4f3 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q49.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q49.native_datafusion/extended.txt @@ -54,11 +54,7 @@ CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : +- CometSubqueryBroadcast - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- ReusedSubquery : : +- CometProject : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.catalog_returns @@ -85,11 +81,7 @@ CometNativeColumnarToRow : : +- CometProject : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : +- CometSubqueryBroadcast - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- ReusedSubquery : +- CometProject : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.store_returns @@ -98,4 +90,4 @@ CometNativeColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.date_dim -Comet accelerated 75 out of 95 eligible operators (78%). Final plan contains 4 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 67 out of 87 eligible operators (77%). Final plan contains 4 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q5.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q5.native_datafusion/extended.txt index adef074558..7fcc516b17 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q5.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q5.native_datafusion/extended.txt @@ -24,11 +24,7 @@ CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.store_returns - : : : +- CometSubqueryBroadcast - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- ReusedSubquery : : +- CometBroadcastExchange : : +- CometProject : : +- CometFilter @@ -48,19 +44,11 @@ CometNativeColumnarToRow : : : :- CometProject : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : +- CometSubqueryBroadcast - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- ReusedSubquery : : : +- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.catalog_returns - : : : +- CometSubqueryBroadcast - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- ReusedSubquery : : +- CometBroadcastExchange : : +- CometProject : : +- CometFilter @@ -80,20 +68,12 @@ CometNativeColumnarToRow : : :- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : +- CometSubqueryBroadcast - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- ReusedSubquery : : +- CometProject : : +- CometBroadcastHashJoin : : :- CometBroadcastExchange : : : +- CometNativeScan parquet spark_catalog.default.web_returns - : : : +- CometSubqueryBroadcast - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- ReusedSubquery : : +- CometProject : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.web_sales @@ -106,4 +86,4 @@ CometNativeColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.web_site -Comet accelerated 100 out of 106 eligible operators (94%). Final plan contains 1 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 80 out of 86 eligible operators (93%). Final plan contains 1 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q51.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q51.native_datafusion/extended.txt index 173d1f064f..f67067735c 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q51.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q51.native_datafusion/extended.txt @@ -43,14 +43,10 @@ TakeOrderedAndProject +- CometBroadcastHashJoin :- CometFilter : +- CometNativeScan parquet spark_catalog.default.store_sales - : +- CometSubqueryBroadcast - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- ReusedSubquery +- CometBroadcastExchange +- CometProject +- CometFilter +- CometNativeScan parquet spark_catalog.default.date_dim -Comet accelerated 42 out of 51 eligible operators (82%). Final plan contains 3 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 38 out of 47 eligible operators (80%). Final plan contains 3 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q54.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q54.native_datafusion/extended.txt index cfb68d7bfb..d262c6d56f 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q54.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q54.native_datafusion/extended.txt @@ -35,11 +35,7 @@ CometNativeColumnarToRow : : : : : : : +- CometProject : : : : : : : +- CometFilter : : : : : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : : : : : +- CometSubqueryBroadcast - : : : : : : : +- CometBroadcastExchange - : : : : : : : +- CometProject - : : : : : : : +- CometFilter - : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : : +- ReusedSubquery : : : : : : +- CometBroadcastExchange : : : : : : +- CometProject : : : : : : +- CometFilter @@ -58,22 +54,8 @@ CometNativeColumnarToRow : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter - : : : : :- Subquery - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometHashAggregate - : : : : : +- CometExchange - : : : : : +- CometHashAggregate - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : +- Subquery - : : : : +- CometNativeColumnarToRow - : : : : +- CometHashAggregate - : : : : +- CometExchange - : : : : +- CometHashAggregate - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : :- ReusedSubquery + : : : : +- ReusedSubquery : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : :- Subquery : : : : +- CometNativeColumnarToRow @@ -122,4 +104,4 @@ CometNativeColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.date_dim -Comet accelerated 105 out of 116 eligible operators (90%). Final plan contains 7 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 89 out of 100 eligible operators (89%). Final plan contains 5 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q56.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q56.native_datafusion/extended.txt index 0744bc307d..e8b3e8e97f 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q56.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q56.native_datafusion/extended.txt @@ -48,11 +48,7 @@ CometNativeColumnarToRow : : : +- CometBroadcastHashJoin : : : :- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : +- CometSubqueryBroadcast - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- ReusedSubquery : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter @@ -81,11 +77,7 @@ CometNativeColumnarToRow : : +- CometBroadcastHashJoin : : :- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : +- CometSubqueryBroadcast - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- ReusedSubquery : : +- CometBroadcastExchange : : +- CometProject : : +- CometFilter @@ -104,4 +96,4 @@ CometNativeColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.item -Comet accelerated 101 out of 104 eligible operators (97%). Final plan contains 1 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 93 out of 96 eligible operators (96%). Final plan contains 1 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q58.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q58.native_datafusion/extended.txt index 4969cdcdfa..6597775c11 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q58.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q58.native_datafusion/extended.txt @@ -23,11 +23,7 @@ CometNativeColumnarToRow : : : : +- CometBroadcastExchange : : : : +- CometProject : : : : +- CometFilter - : : : : : +- Subquery - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- ReusedSubquery : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : +- Subquery : : : : +- CometNativeColumnarToRow @@ -64,26 +60,7 @@ CometNativeColumnarToRow : : +- CometBroadcastHashJoin : : :- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : +- CometSubqueryBroadcast - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : : +- Subquery - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- Subquery - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- ReusedSubquery : : +- CometBroadcastExchange : : +- CometProject : : +- CometFilter @@ -114,26 +91,7 @@ CometNativeColumnarToRow : +- CometBroadcastHashJoin : :- CometFilter : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : +- CometSubqueryBroadcast - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : : +- Subquery - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- Subquery - : : +- CometNativeColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- ReusedSubquery : +- CometBroadcastExchange : +- CometProject : +- CometFilter @@ -154,4 +112,4 @@ CometNativeColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.date_dim -Comet accelerated 130 out of 145 eligible operators (89%). Final plan contains 10 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 97 out of 108 eligible operators (89%). Final plan contains 5 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q6.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q6.native_datafusion/extended.txt index e8759a6e10..dd68f2476e 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q6.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q6.native_datafusion/extended.txt @@ -25,14 +25,7 @@ CometNativeColumnarToRow : : +- CometBroadcastExchange : : +- CometProject : : +- CometFilter - : : : +- Subquery - : : : +- CometNativeColumnarToRow - : : : +- CometHashAggregate - : : : +- CometExchange - : : : +- CometHashAggregate - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- ReusedSubquery : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- Subquery : : +- CometNativeColumnarToRow @@ -69,4 +62,4 @@ CometNativeColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.item -Comet accelerated 61 out of 66 eligible operators (92%). Final plan contains 4 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 55 out of 60 eligible operators (91%). Final plan contains 3 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q60.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q60.native_datafusion/extended.txt index 0744bc307d..e8b3e8e97f 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q60.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q60.native_datafusion/extended.txt @@ -48,11 +48,7 @@ CometNativeColumnarToRow : : : +- CometBroadcastHashJoin : : : :- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : +- CometSubqueryBroadcast - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- ReusedSubquery : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter @@ -81,11 +77,7 @@ CometNativeColumnarToRow : : +- CometBroadcastHashJoin : : :- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : +- CometSubqueryBroadcast - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- ReusedSubquery : : +- CometBroadcastExchange : : +- CometProject : : +- CometFilter @@ -104,4 +96,4 @@ CometNativeColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.item -Comet accelerated 101 out of 104 eligible operators (97%). Final plan contains 1 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 93 out of 96 eligible operators (96%). Final plan contains 1 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q61.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q61.native_datafusion/extended.txt index 8d91cbe7a2..39f985df28 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q61.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q61.native_datafusion/extended.txt @@ -63,11 +63,7 @@ Project : : : : +- CometBroadcastHashJoin : : : : :- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : : : +- CometSubqueryBroadcast - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- ReusedSubquery : : : : +- CometBroadcastExchange : : : : +- CometProject : : : : +- CometFilter @@ -88,4 +84,4 @@ Project +- CometFilter +- CometNativeScan parquet spark_catalog.default.item -Comet accelerated 82 out of 87 eligible operators (94%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 78 out of 83 eligible operators (93%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q65.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q65.native_datafusion/extended.txt index 21d60497a1..b3937f6c97 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q65.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q65.native_datafusion/extended.txt @@ -42,14 +42,10 @@ CometNativeColumnarToRow +- CometBroadcastHashJoin :- CometFilter : +- CometNativeScan parquet spark_catalog.default.store_sales - : +- CometSubqueryBroadcast - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- ReusedSubquery +- CometBroadcastExchange +- CometProject +- CometFilter +- CometNativeScan parquet spark_catalog.default.date_dim -Comet accelerated 50 out of 52 eligible operators (96%). Final plan contains 1 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 46 out of 48 eligible operators (95%). Final plan contains 1 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q66.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q66.native_datafusion/extended.txt index c07ecfe46a..3b8d1937f2 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q66.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q66.native_datafusion/extended.txt @@ -49,10 +49,7 @@ CometNativeColumnarToRow : : : +- CometBroadcastHashJoin : : : :- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : +- CometSubqueryBroadcast - : : : : +- CometBroadcastExchange - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- ReusedSubquery : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter @@ -69,4 +66,4 @@ CometNativeColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.ship_mode -Comet accelerated 67 out of 69 eligible operators (97%). Final plan contains 1 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 64 out of 66 eligible operators (96%). Final plan contains 1 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q69.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q69.native_datafusion/extended.txt index 6d398580fb..cb518a5c47 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q69.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q69.native_datafusion/extended.txt @@ -32,11 +32,7 @@ TakeOrderedAndProject : : : +- CometProject : : : +- CometBroadcastHashJoin : : : :- CometNativeScan parquet spark_catalog.default.web_sales - : : : : +- CometSubqueryBroadcast - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- ReusedSubquery : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter @@ -46,11 +42,7 @@ TakeOrderedAndProject : : +- CometProject : : +- CometBroadcastHashJoin : : :- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : +- CometSubqueryBroadcast - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- ReusedSubquery : : +- CometBroadcastExchange : : +- CometProject : : +- CometFilter @@ -66,4 +58,4 @@ TakeOrderedAndProject +- CometFilter +- CometNativeScan parquet spark_catalog.default.customer_demographics -Comet accelerated 44 out of 61 eligible operators (72%). Final plan contains 6 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 36 out of 53 eligible operators (67%). Final plan contains 6 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q70.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q70.native_datafusion/extended.txt index d1ff1b903e..173923495f 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q70.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q70.native_datafusion/extended.txt @@ -47,11 +47,7 @@ TakeOrderedAndProject : +- CometBroadcastHashJoin : :- CometFilter : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : +- CometSubqueryBroadcast - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- ReusedSubquery : +- CometBroadcastExchange : +- CometProject : +- CometFilter @@ -61,4 +57,4 @@ TakeOrderedAndProject +- CometFilter +- CometNativeScan parquet spark_catalog.default.date_dim -Comet accelerated 39 out of 57 eligible operators (68%). Final plan contains 5 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 35 out of 53 eligible operators (66%). Final plan contains 5 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q71.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q71.native_datafusion/extended.txt index 834098d4ee..2c056d01d7 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q71.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q71.native_datafusion/extended.txt @@ -30,11 +30,7 @@ CometNativeColumnarToRow : : +- CometBroadcastHashJoin : : :- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : +- CometSubqueryBroadcast - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- ReusedSubquery : : +- CometBroadcastExchange : : +- CometProject : : +- CometFilter @@ -43,11 +39,7 @@ CometNativeColumnarToRow : +- CometBroadcastHashJoin : :- CometFilter : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : +- CometSubqueryBroadcast - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- ReusedSubquery : +- CometBroadcastExchange : +- CometProject : +- CometFilter @@ -57,4 +49,4 @@ CometNativeColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.time_dim -Comet accelerated 54 out of 57 eligible operators (94%). Final plan contains 1 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 46 out of 49 eligible operators (93%). Final plan contains 1 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q74.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q74.native_datafusion/extended.txt index 7768138645..cfe634568f 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q74.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q74.native_datafusion/extended.txt @@ -62,10 +62,7 @@ CometNativeColumnarToRow : : +- CometBroadcastExchange : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : +- CometSubqueryBroadcast - : : +- CometBroadcastExchange - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- ReusedSubquery : +- CometBroadcastExchange : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -83,12 +80,9 @@ CometNativeColumnarToRow : +- CometBroadcastExchange : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.web_sales - : +- CometSubqueryBroadcast - : +- CometBroadcastExchange - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- ReusedSubquery +- CometBroadcastExchange +- CometFilter +- CometNativeScan parquet spark_catalog.default.date_dim -Comet accelerated 87 out of 91 eligible operators (95%). Final plan contains 1 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 81 out of 85 eligible operators (95%). Final plan contains 1 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q75.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q75.native_datafusion/extended.txt index ee7c0af216..bccd23b141 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q75.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q75.native_datafusion/extended.txt @@ -48,10 +48,7 @@ CometNativeColumnarToRow : : : : +- CometBroadcastHashJoin : : : : :- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : : : +- CometSubqueryBroadcast - : : : : : +- CometBroadcastExchange - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- ReusedSubquery : : : : +- CometBroadcastExchange : : : : +- CometProject : : : : +- CometFilter @@ -74,10 +71,7 @@ CometNativeColumnarToRow : : : +- CometBroadcastHashJoin : : : :- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : : +- CometSubqueryBroadcast - : : : : +- CometBroadcastExchange - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- ReusedSubquery : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter @@ -136,10 +130,7 @@ CometNativeColumnarToRow : : : +- CometBroadcastHashJoin : : : :- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : : +- CometSubqueryBroadcast - : : : : +- CometBroadcastExchange - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- ReusedSubquery : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter @@ -162,10 +153,7 @@ CometNativeColumnarToRow : : +- CometBroadcastHashJoin : : :- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : +- CometSubqueryBroadcast - : : : +- CometBroadcastExchange - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- ReusedSubquery : : +- CometBroadcastExchange : : +- CometProject : : +- CometFilter @@ -179,4 +167,4 @@ CometNativeColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.web_returns -Comet accelerated 173 out of 179 eligible operators (96%). Final plan contains 1 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 161 out of 167 eligible operators (96%). Final plan contains 1 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q77.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q77.native_datafusion/extended.txt index 8805a2e6c2..38ee51e753 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q77.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q77.native_datafusion/extended.txt @@ -39,11 +39,7 @@ TakeOrderedAndProject : : +- CometBroadcastHashJoin : : :- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.store_returns - : : : +- CometSubqueryBroadcast - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- ReusedSubquery : : +- CometBroadcastExchange : : +- CometProject : : +- CometFilter @@ -61,11 +57,7 @@ TakeOrderedAndProject : : +- CometProject : : +- CometBroadcastHashJoin : : :- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : +- CometSubqueryBroadcast - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- ReusedSubquery : : +- CometBroadcastExchange : : +- CometProject : : +- CometFilter @@ -77,11 +69,7 @@ TakeOrderedAndProject : +- CometProject : +- CometBroadcastHashJoin : :- CometNativeScan parquet spark_catalog.default.catalog_returns - : : +- CometSubqueryBroadcast - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- ReusedSubquery : +- CometBroadcastExchange : +- CometProject : +- CometFilter @@ -98,11 +86,7 @@ TakeOrderedAndProject : : +- CometBroadcastHashJoin : : :- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : +- CometSubqueryBroadcast - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- ReusedSubquery : : +- CometBroadcastExchange : : +- CometProject : : +- CometFilter @@ -120,11 +104,7 @@ TakeOrderedAndProject : +- CometBroadcastHashJoin : :- CometFilter : : +- CometNativeScan parquet spark_catalog.default.web_returns - : : +- CometSubqueryBroadcast - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- ReusedSubquery : +- CometBroadcastExchange : +- CometProject : +- CometFilter @@ -133,4 +113,4 @@ TakeOrderedAndProject +- CometFilter +- CometNativeScan parquet spark_catalog.default.web_page -Comet accelerated 115 out of 129 eligible operators (89%). Final plan contains 5 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 95 out of 109 eligible operators (87%). Final plan contains 5 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q78.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q78.native_datafusion/extended.txt index 8426b9653c..d29dbc13e5 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q78.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q78.native_datafusion/extended.txt @@ -43,10 +43,7 @@ TakeOrderedAndProject : : : +- CometExchange : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : +- CometSubqueryBroadcast - : : : +- CometBroadcastExchange - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- ReusedSubquery : : +- CometSort : : +- CometExchange : : +- CometProject @@ -69,10 +66,7 @@ TakeOrderedAndProject : : +- CometExchange : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : +- CometSubqueryBroadcast - : : +- CometBroadcastExchange - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- ReusedSubquery : +- CometSort : +- CometExchange : +- CometProject @@ -82,4 +76,4 @@ TakeOrderedAndProject +- CometFilter +- CometNativeScan parquet spark_catalog.default.date_dim -Comet accelerated 77 out of 82 eligible operators (93%). Final plan contains 1 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 71 out of 76 eligible operators (93%). Final plan contains 1 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q80.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q80.native_datafusion/extended.txt index 27e32a1602..9f67d9b8b5 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q80.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q80.native_datafusion/extended.txt @@ -65,11 +65,7 @@ CometNativeColumnarToRow : : : : : : +- CometExchange : : : : : : +- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : : : +- CometSubqueryBroadcast - : : : : : : +- CometBroadcastExchange - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : +- ReusedSubquery : : : : : +- CometSort : : : : : +- CometExchange : : : : : +- CometProject @@ -108,11 +104,7 @@ CometNativeColumnarToRow : : : : : +- CometExchange : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : : : +- CometSubqueryBroadcast - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- ReusedSubquery : : : : +- CometSort : : : : +- CometExchange : : : : +- CometProject @@ -135,4 +127,4 @@ CometNativeColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.promotion -Comet accelerated 132 out of 135 eligible operators (97%). Final plan contains 1 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 124 out of 127 eligible operators (97%). Final plan contains 1 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q81.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q81.native_datafusion/extended.txt index 6d9aa9697d..f10339641b 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q81.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q81.native_datafusion/extended.txt @@ -43,11 +43,7 @@ CometNativeColumnarToRow : : : +- CometBroadcastHashJoin : : : :- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.catalog_returns - : : : : +- CometSubqueryBroadcast - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- ReusedSubquery : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter @@ -65,4 +61,4 @@ CometNativeColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.customer_address -Comet accelerated 63 out of 65 eligible operators (96%). Final plan contains 1 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 59 out of 61 eligible operators (96%). Final plan contains 1 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q83.ansi.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q83.ansi.native_datafusion/extended.txt index e4d1992492..32adaba671 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q83.ansi.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q83.ansi.native_datafusion/extended.txt @@ -54,20 +54,7 @@ CometNativeColumnarToRow : : +- CometBroadcastHashJoin : : :- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.catalog_returns - : : : +- CometSubqueryBroadcast - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- ReusedSubquery : : +- CometBroadcastExchange : : +- CometProject : : +- CometFilter @@ -95,20 +82,7 @@ CometNativeColumnarToRow : +- CometBroadcastHashJoin : :- CometFilter : : +- CometNativeScan parquet spark_catalog.default.web_returns - : : +- CometSubqueryBroadcast - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometNativeScan parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- ReusedSubquery : +- CometBroadcastExchange : +- CometProject : +- CometFilter @@ -127,4 +101,4 @@ CometNativeColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.date_dim -Comet accelerated 124 out of 127 eligible operators (97%). Final plan contains 1 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 98 out of 101 eligible operators (97%). Final plan contains 1 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q87.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q87.native_datafusion/extended.txt index 5c80165c1f..7680e50295 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q87.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q87.native_datafusion/extended.txt @@ -39,11 +39,7 @@ HashAggregate : : +- CometBroadcastHashJoin : : :- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : +- CometSubqueryBroadcast - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- ReusedSubquery : : +- CometBroadcastExchange : : +- CometProject : : +- CometFilter @@ -63,11 +59,7 @@ HashAggregate : +- CometBroadcastHashJoin : :- CometFilter : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : +- CometSubqueryBroadcast - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- ReusedSubquery : +- CometBroadcastExchange : +- CometProject : +- CometFilter @@ -77,4 +69,4 @@ HashAggregate +- CometFilter +- CometNativeScan parquet spark_catalog.default.customer -Comet accelerated 64 out of 74 eligible operators (86%). Final plan contains 4 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 56 out of 66 eligible operators (84%). Final plan contains 4 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q92.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q92.native_datafusion/extended.txt index 40926fafac..bcf7db282b 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q92.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q92.native_datafusion/extended.txt @@ -28,11 +28,7 @@ CometNativeColumnarToRow : +- CometBroadcastHashJoin : :- CometFilter : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : +- CometSubqueryBroadcast - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- ReusedSubquery : +- CometBroadcastExchange : +- CometProject : +- CometFilter @@ -42,4 +38,4 @@ CometNativeColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.date_dim -Comet accelerated 40 out of 42 eligible operators (95%). Final plan contains 1 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 36 out of 38 eligible operators (94%). Final plan contains 1 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q97.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q97.native_datafusion/extended.txt index 74163dff2b..a56bd059fa 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q97.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q97.native_datafusion/extended.txt @@ -27,14 +27,10 @@ CometNativeColumnarToRow +- CometProject +- CometBroadcastHashJoin :- CometNativeScan parquet spark_catalog.default.catalog_sales - : +- CometSubqueryBroadcast - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- ReusedSubquery +- CometBroadcastExchange +- CometProject +- CometFilter +- CometNativeScan parquet spark_catalog.default.date_dim -Comet accelerated 35 out of 37 eligible operators (94%). Final plan contains 1 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 31 out of 33 eligible operators (93%). Final plan contains 1 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q1.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q1.native_datafusion/extended.txt index 5875fa3a5e..f49109dafb 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q1.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q1.native_datafusion/extended.txt @@ -35,11 +35,7 @@ CometNativeColumnarToRow : : +- CometBroadcastHashJoin : : :- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.store_returns - : : : +- CometSubqueryBroadcast - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- ReusedSubquery : : +- CometBroadcastExchange : : +- CometProject : : +- CometFilter @@ -53,4 +49,4 @@ CometNativeColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.customer -Comet accelerated 51 out of 53 eligible operators (96%). Final plan contains 1 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 47 out of 49 eligible operators (95%). Final plan contains 1 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q10.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q10.native_datafusion/extended.txt index 2e79f6f464..b7c8209556 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q10.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q10.native_datafusion/extended.txt @@ -33,11 +33,7 @@ TakeOrderedAndProject : : : +- CometProject : : : +- CometBroadcastHashJoin : : : :- CometNativeScan parquet spark_catalog.default.web_sales - : : : : +- CometSubqueryBroadcast - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- ReusedSubquery : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter @@ -47,11 +43,7 @@ TakeOrderedAndProject : : +- CometProject : : +- CometBroadcastHashJoin : : :- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : +- CometSubqueryBroadcast - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- ReusedSubquery : : +- CometBroadcastExchange : : +- CometProject : : +- CometFilter @@ -67,4 +59,4 @@ TakeOrderedAndProject +- CometFilter +- CometNativeScan parquet spark_catalog.default.customer_demographics -Comet accelerated 44 out of 62 eligible operators (70%). Final plan contains 6 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 36 out of 54 eligible operators (66%). Final plan contains 6 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q11.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q11.native_datafusion/extended.txt index edda694b73..7f06caa2e1 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q11.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q11.native_datafusion/extended.txt @@ -63,10 +63,7 @@ CometNativeColumnarToRow : : +- CometBroadcastExchange : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : +- CometSubqueryBroadcast - : : +- CometBroadcastExchange - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- ReusedSubquery : +- CometBroadcastExchange : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -84,12 +81,9 @@ CometNativeColumnarToRow : +- CometBroadcastExchange : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.web_sales - : +- CometSubqueryBroadcast - : +- CometBroadcastExchange - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- ReusedSubquery +- CometBroadcastExchange +- CometFilter +- CometNativeScan parquet spark_catalog.default.date_dim -Comet accelerated 88 out of 92 eligible operators (95%). Final plan contains 1 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 82 out of 86 eligible operators (95%). Final plan contains 1 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14a.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14a.native_datafusion/extended.txt index d2a44e9c52..1ae3f23b40 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14a.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14a.native_datafusion/extended.txt @@ -16,11 +16,7 @@ CometNativeColumnarToRow : : :- CometProject : : : +- CometBroadcastHashJoin : : : :- CometNativeScan parquet spark_catalog.default.store_sales - : : : : +- CometSubqueryBroadcast - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- ReusedSubquery : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter @@ -28,11 +24,7 @@ CometNativeColumnarToRow : : :- CometProject : : : +- CometBroadcastHashJoin : : : :- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : +- CometSubqueryBroadcast - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- ReusedSubquery : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter @@ -40,11 +32,7 @@ CometNativeColumnarToRow : : +- CometProject : : +- CometBroadcastHashJoin : : :- CometNativeScan parquet spark_catalog.default.web_sales - : : : +- CometSubqueryBroadcast - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- ReusedSubquery : : +- CometBroadcastExchange : : +- CometProject : : +- CometFilter @@ -96,11 +84,7 @@ CometNativeColumnarToRow : : : : : : +- CometBroadcastHashJoin : : : : : : :- CometFilter : : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : : : : +- CometSubqueryBroadcast - : : : : : : : +- CometBroadcastExchange - : : : : : : : +- CometProject - : : : : : : : +- CometFilter - : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : : +- ReusedSubquery : : : : : : +- CometBroadcastExchange : : : : : : +- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.item @@ -119,11 +103,7 @@ CometNativeColumnarToRow : : : : +- CometBroadcastHashJoin : : : : :- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : : : +- CometSubqueryBroadcast - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- ReusedSubquery : : : : +- CometBroadcastExchange : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.item @@ -167,11 +147,7 @@ CometNativeColumnarToRow : : : : : +- CometBroadcastHashJoin : : : : : :- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : : : +- CometSubqueryBroadcast - : : : : : : +- CometBroadcastExchange - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : +- ReusedSubquery : : : : : +- CometBroadcastExchange : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.item @@ -190,11 +166,7 @@ CometNativeColumnarToRow : : : +- CometBroadcastHashJoin : : : :- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : : +- CometSubqueryBroadcast - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- ReusedSubquery : : : +- CometBroadcastExchange : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.item @@ -219,11 +191,7 @@ CometNativeColumnarToRow : : :- CometBroadcastHashJoin : : : :- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : +- CometSubqueryBroadcast - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- ReusedSubquery : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometBroadcastHashJoin @@ -256,11 +224,7 @@ CometNativeColumnarToRow : : : : : : +- CometBroadcastHashJoin : : : : : : :- CometFilter : : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : : : : +- CometSubqueryBroadcast - : : : : : : : +- CometBroadcastExchange - : : : : : : : +- CometProject - : : : : : : : +- CometFilter - : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : : +- ReusedSubquery : : : : : : +- CometBroadcastExchange : : : : : : +- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.item @@ -279,11 +243,7 @@ CometNativeColumnarToRow : : : : +- CometBroadcastHashJoin : : : : :- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : : : +- CometSubqueryBroadcast - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- ReusedSubquery : : : : +- CometBroadcastExchange : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.item @@ -327,11 +287,7 @@ CometNativeColumnarToRow : : : : : +- CometBroadcastHashJoin : : : : : :- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : : : +- CometSubqueryBroadcast - : : : : : : +- CometBroadcastExchange - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : +- ReusedSubquery : : : : : +- CometBroadcastExchange : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.item @@ -350,11 +306,7 @@ CometNativeColumnarToRow : : : +- CometBroadcastHashJoin : : : :- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : : +- CometSubqueryBroadcast - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- ReusedSubquery : : : +- CometBroadcastExchange : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.item @@ -379,11 +331,7 @@ CometNativeColumnarToRow : :- CometBroadcastHashJoin : : :- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : +- CometSubqueryBroadcast - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- ReusedSubquery : : +- CometBroadcastExchange : : +- CometProject : : +- CometBroadcastHashJoin @@ -416,11 +364,7 @@ CometNativeColumnarToRow : : : : : +- CometBroadcastHashJoin : : : : : :- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : : : +- CometSubqueryBroadcast - : : : : : : +- CometBroadcastExchange - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : +- ReusedSubquery : : : : : +- CometBroadcastExchange : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.item @@ -439,11 +383,7 @@ CometNativeColumnarToRow : : : +- CometBroadcastHashJoin : : : :- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : : +- CometSubqueryBroadcast - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- ReusedSubquery : : : +- CometBroadcastExchange : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.item @@ -487,11 +427,7 @@ CometNativeColumnarToRow : : : : +- CometBroadcastHashJoin : : : : :- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : : +- CometSubqueryBroadcast - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- ReusedSubquery : : : : +- CometBroadcastExchange : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.item @@ -510,11 +446,7 @@ CometNativeColumnarToRow : : +- CometBroadcastHashJoin : : :- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : +- CometSubqueryBroadcast - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- ReusedSubquery : : +- CometBroadcastExchange : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.item @@ -527,4 +459,4 @@ CometNativeColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.date_dim -Comet accelerated 499 out of 526 eligible operators (94%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 431 out of 458 eligible operators (94%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14b.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14b.native_datafusion/extended.txt index 2d20c40e4b..1b9b46662f 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14b.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14b.native_datafusion/extended.txt @@ -11,11 +11,7 @@ CometNativeColumnarToRow : : :- CometProject : : : +- CometBroadcastHashJoin : : : :- CometNativeScan parquet spark_catalog.default.store_sales - : : : : +- CometSubqueryBroadcast - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- ReusedSubquery : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter @@ -23,11 +19,7 @@ CometNativeColumnarToRow : : :- CometProject : : : +- CometBroadcastHashJoin : : : :- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : +- CometSubqueryBroadcast - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- ReusedSubquery : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter @@ -35,11 +27,7 @@ CometNativeColumnarToRow : : +- CometProject : : +- CometBroadcastHashJoin : : :- CometNativeScan parquet spark_catalog.default.web_sales - : : : +- CometSubqueryBroadcast - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- ReusedSubquery : : +- CometBroadcastExchange : : +- CometProject : : +- CometFilter @@ -96,11 +84,7 @@ CometNativeColumnarToRow : : : : : : +- CometBroadcastHashJoin : : : : : : :- CometFilter : : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : : : : +- CometSubqueryBroadcast - : : : : : : : +- CometBroadcastExchange - : : : : : : : +- CometProject - : : : : : : : +- CometFilter - : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : : +- ReusedSubquery : : : : : : +- CometBroadcastExchange : : : : : : +- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.item @@ -119,11 +103,7 @@ CometNativeColumnarToRow : : : : +- CometBroadcastHashJoin : : : : :- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : : : +- CometSubqueryBroadcast - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- ReusedSubquery : : : : +- CometBroadcastExchange : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.item @@ -167,11 +147,7 @@ CometNativeColumnarToRow : : : : : +- CometBroadcastHashJoin : : : : : :- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : : : +- CometSubqueryBroadcast - : : : : : : +- CometBroadcastExchange - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : +- ReusedSubquery : : : : : +- CometBroadcastExchange : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.item @@ -190,11 +166,7 @@ CometNativeColumnarToRow : : : +- CometBroadcastHashJoin : : : :- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : : +- CometSubqueryBroadcast - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- ReusedSubquery : : : +- CometBroadcastExchange : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.item @@ -266,11 +238,7 @@ CometNativeColumnarToRow : : : : : +- CometBroadcastHashJoin : : : : : :- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : : : +- CometSubqueryBroadcast - : : : : : : +- CometBroadcastExchange - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : +- ReusedSubquery : : : : : +- CometBroadcastExchange : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.item @@ -289,11 +257,7 @@ CometNativeColumnarToRow : : : +- CometBroadcastHashJoin : : : :- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : : +- CometSubqueryBroadcast - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- ReusedSubquery : : : +- CometBroadcastExchange : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.item @@ -337,11 +301,7 @@ CometNativeColumnarToRow : : : : +- CometBroadcastHashJoin : : : : :- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : : +- CometSubqueryBroadcast - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- ReusedSubquery : : : : +- CometBroadcastExchange : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.item @@ -360,11 +320,7 @@ CometNativeColumnarToRow : : +- CometBroadcastHashJoin : : :- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : +- CometSubqueryBroadcast - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- ReusedSubquery : : +- CometBroadcastExchange : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.item @@ -382,4 +338,4 @@ CometNativeColumnarToRow : +- CometNativeScan parquet spark_catalog.default.date_dim +- CometNativeScan parquet spark_catalog.default.date_dim -Comet accelerated 354 out of 377 eligible operators (93%). Final plan contains 6 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 310 out of 333 eligible operators (93%). Final plan contains 6 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q17.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q17.native_datafusion/extended.txt index 2650cd8f1a..a3f43554cf 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q17.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q17.native_datafusion/extended.txt @@ -35,11 +35,7 @@ CometNativeColumnarToRow : : : : : +- CometBroadcastExchange : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : : +- CometSubqueryBroadcast - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- ReusedSubquery : : : : +- CometBroadcastExchange : : : : +- CometProject : : : : +- CometFilter @@ -61,4 +57,4 @@ CometNativeColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.item -Comet accelerated 58 out of 61 eligible operators (95%). Final plan contains 1 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 54 out of 57 eligible operators (94%). Final plan contains 1 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q23a.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q23a.native_datafusion/extended.txt index 3e46e4a329..6dd0348345 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q23a.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q23a.native_datafusion/extended.txt @@ -94,11 +94,7 @@ CometNativeColumnarToRow : : +- CometProject : : +- CometBroadcastHashJoin : : :- CometNativeScan parquet spark_catalog.default.web_sales - : : : +- CometSubqueryBroadcast - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- ReusedSubquery : : +- CometBroadcastExchange : : +- CometProject : : +- CometFilter @@ -143,4 +139,4 @@ CometNativeColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.date_dim -Comet accelerated 135 out of 142 eligible operators (95%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 131 out of 138 eligible operators (94%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q23b.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q23b.native_datafusion/extended.txt index 79baaf1aa9..f2228c277c 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q23b.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q23b.native_datafusion/extended.txt @@ -124,11 +124,7 @@ CometNativeColumnarToRow : : : +- CometBroadcastHashJoin : : : :- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : : +- CometSubqueryBroadcast - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- ReusedSubquery : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter @@ -195,4 +191,4 @@ CometNativeColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.date_dim -Comet accelerated 185 out of 194 eligible operators (95%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 181 out of 190 eligible operators (95%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q25.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q25.native_datafusion/extended.txt index 2650cd8f1a..a3f43554cf 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q25.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q25.native_datafusion/extended.txt @@ -35,11 +35,7 @@ CometNativeColumnarToRow : : : : : +- CometBroadcastExchange : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : : +- CometSubqueryBroadcast - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- ReusedSubquery : : : : +- CometBroadcastExchange : : : : +- CometProject : : : : +- CometFilter @@ -61,4 +57,4 @@ CometNativeColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.item -Comet accelerated 58 out of 61 eligible operators (95%). Final plan contains 1 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 54 out of 57 eligible operators (94%). Final plan contains 1 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q30.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q30.native_datafusion/extended.txt index 2a22518df4..3f88234deb 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q30.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q30.native_datafusion/extended.txt @@ -43,11 +43,7 @@ CometNativeColumnarToRow : : : +- CometBroadcastHashJoin : : : :- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.web_returns - : : : : +- CometSubqueryBroadcast - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- ReusedSubquery : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter @@ -65,4 +61,4 @@ CometNativeColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.customer_address -Comet accelerated 63 out of 65 eligible operators (96%). Final plan contains 1 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 59 out of 61 eligible operators (96%). Final plan contains 1 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q31.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q31.native_datafusion/extended.txt index bf4fb2aa77..188e553f1a 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q31.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q31.native_datafusion/extended.txt @@ -78,10 +78,7 @@ CometNativeColumnarToRow : : : +- CometBroadcastHashJoin : : : :- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : : +- CometSubqueryBroadcast - : : : : +- CometBroadcastExchange - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- ReusedSubquery : : : +- CometBroadcastExchange : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -98,10 +95,7 @@ CometNativeColumnarToRow : : +- CometBroadcastHashJoin : : :- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : +- CometSubqueryBroadcast - : : : +- CometBroadcastExchange - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- ReusedSubquery : : +- CometBroadcastExchange : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -118,10 +112,7 @@ CometNativeColumnarToRow : +- CometBroadcastHashJoin : :- CometFilter : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : +- CometSubqueryBroadcast - : : +- CometBroadcastExchange - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- ReusedSubquery : +- CometBroadcastExchange : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -129,4 +120,4 @@ CometNativeColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.customer_address -Comet accelerated 123 out of 129 eligible operators (95%). Final plan contains 1 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 114 out of 120 eligible operators (95%). Final plan contains 1 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q32.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q32.native_datafusion/extended.txt index db07e648bc..a536c4b38f 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q32.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q32.native_datafusion/extended.txt @@ -28,11 +28,7 @@ CometNativeColumnarToRow : +- CometBroadcastHashJoin : :- CometFilter : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : +- CometSubqueryBroadcast - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- ReusedSubquery : +- CometBroadcastExchange : +- CometProject : +- CometFilter @@ -42,4 +38,4 @@ CometNativeColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.date_dim -Comet accelerated 40 out of 42 eligible operators (95%). Final plan contains 1 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 36 out of 38 eligible operators (94%). Final plan contains 1 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q33.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q33.native_datafusion/extended.txt index d64ba66952..9f17b73589 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q33.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q33.native_datafusion/extended.txt @@ -47,11 +47,7 @@ CometNativeColumnarToRow : : : +- CometBroadcastHashJoin : : : :- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : +- CometSubqueryBroadcast - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- ReusedSubquery : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter @@ -79,11 +75,7 @@ CometNativeColumnarToRow : : +- CometBroadcastHashJoin : : :- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : +- CometSubqueryBroadcast - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- ReusedSubquery : : +- CometBroadcastExchange : : +- CometProject : : +- CometFilter @@ -101,4 +93,4 @@ CometNativeColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.item -Comet accelerated 98 out of 101 eligible operators (97%). Final plan contains 1 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 90 out of 93 eligible operators (96%). Final plan contains 1 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q35.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q35.native_datafusion/extended.txt index 2e79f6f464..b7c8209556 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q35.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q35.native_datafusion/extended.txt @@ -33,11 +33,7 @@ TakeOrderedAndProject : : : +- CometProject : : : +- CometBroadcastHashJoin : : : :- CometNativeScan parquet spark_catalog.default.web_sales - : : : : +- CometSubqueryBroadcast - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- ReusedSubquery : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter @@ -47,11 +43,7 @@ TakeOrderedAndProject : : +- CometProject : : +- CometBroadcastHashJoin : : :- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : +- CometSubqueryBroadcast - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- ReusedSubquery : : +- CometBroadcastExchange : : +- CometProject : : +- CometFilter @@ -67,4 +59,4 @@ TakeOrderedAndProject +- CometFilter +- CometNativeScan parquet spark_catalog.default.customer_demographics -Comet accelerated 44 out of 62 eligible operators (70%). Final plan contains 6 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 36 out of 54 eligible operators (66%). Final plan contains 6 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q38.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q38.native_datafusion/extended.txt index c0314d8fb0..c965be9cc3 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q38.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q38.native_datafusion/extended.txt @@ -37,11 +37,7 @@ CometNativeColumnarToRow : : +- CometBroadcastHashJoin : : :- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : +- CometSubqueryBroadcast - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- ReusedSubquery : : +- CometBroadcastExchange : : +- CometProject : : +- CometFilter @@ -60,11 +56,7 @@ CometNativeColumnarToRow : +- CometBroadcastHashJoin : :- CometFilter : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : +- CometSubqueryBroadcast - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- ReusedSubquery : +- CometBroadcastExchange : +- CometProject : +- CometFilter @@ -74,4 +66,4 @@ CometNativeColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.customer -Comet accelerated 71 out of 74 eligible operators (95%). Final plan contains 1 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 63 out of 66 eligible operators (95%). Final plan contains 1 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q4.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q4.native_datafusion/extended.txt index bf61cfd81e..ff66588e1a 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q4.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q4.native_datafusion/extended.txt @@ -66,10 +66,7 @@ CometNativeColumnarToRow : : : : +- CometBroadcastExchange : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : +- CometSubqueryBroadcast - : : : : +- CometBroadcastExchange - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- ReusedSubquery : : : +- CometBroadcastExchange : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -87,10 +84,7 @@ CometNativeColumnarToRow : : : +- CometBroadcastExchange : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : +- CometSubqueryBroadcast - : : : +- CometBroadcastExchange - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- ReusedSubquery : : +- CometBroadcastExchange : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -109,10 +103,7 @@ CometNativeColumnarToRow : : +- CometBroadcastExchange : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : +- CometSubqueryBroadcast - : : +- CometBroadcastExchange - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- ReusedSubquery : +- CometBroadcastExchange : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -130,12 +121,9 @@ CometNativeColumnarToRow : +- CometBroadcastExchange : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.web_sales - : +- CometSubqueryBroadcast - : +- CometBroadcastExchange - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- ReusedSubquery +- CometBroadcastExchange +- CometFilter +- CometNativeScan parquet spark_catalog.default.date_dim -Comet accelerated 132 out of 138 eligible operators (95%). Final plan contains 1 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 120 out of 126 eligible operators (95%). Final plan contains 1 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q49.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q49.native_datafusion/extended.txt index f2ae44fb86..cd6c39e4f3 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q49.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q49.native_datafusion/extended.txt @@ -54,11 +54,7 @@ CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : +- CometSubqueryBroadcast - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- ReusedSubquery : : +- CometProject : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.catalog_returns @@ -85,11 +81,7 @@ CometNativeColumnarToRow : : +- CometProject : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : +- CometSubqueryBroadcast - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- ReusedSubquery : +- CometProject : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.store_returns @@ -98,4 +90,4 @@ CometNativeColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.date_dim -Comet accelerated 75 out of 95 eligible operators (78%). Final plan contains 4 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 67 out of 87 eligible operators (77%). Final plan contains 4 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q5.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q5.native_datafusion/extended.txt index adef074558..7fcc516b17 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q5.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q5.native_datafusion/extended.txt @@ -24,11 +24,7 @@ CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.store_returns - : : : +- CometSubqueryBroadcast - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- ReusedSubquery : : +- CometBroadcastExchange : : +- CometProject : : +- CometFilter @@ -48,19 +44,11 @@ CometNativeColumnarToRow : : : :- CometProject : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : +- CometSubqueryBroadcast - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- ReusedSubquery : : : +- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.catalog_returns - : : : +- CometSubqueryBroadcast - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- ReusedSubquery : : +- CometBroadcastExchange : : +- CometProject : : +- CometFilter @@ -80,20 +68,12 @@ CometNativeColumnarToRow : : :- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : +- CometSubqueryBroadcast - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- ReusedSubquery : : +- CometProject : : +- CometBroadcastHashJoin : : :- CometBroadcastExchange : : : +- CometNativeScan parquet spark_catalog.default.web_returns - : : : +- CometSubqueryBroadcast - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- ReusedSubquery : : +- CometProject : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.web_sales @@ -106,4 +86,4 @@ CometNativeColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.web_site -Comet accelerated 100 out of 106 eligible operators (94%). Final plan contains 1 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 80 out of 86 eligible operators (93%). Final plan contains 1 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q51.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q51.native_datafusion/extended.txt index 173d1f064f..f67067735c 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q51.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q51.native_datafusion/extended.txt @@ -43,14 +43,10 @@ TakeOrderedAndProject +- CometBroadcastHashJoin :- CometFilter : +- CometNativeScan parquet spark_catalog.default.store_sales - : +- CometSubqueryBroadcast - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- ReusedSubquery +- CometBroadcastExchange +- CometProject +- CometFilter +- CometNativeScan parquet spark_catalog.default.date_dim -Comet accelerated 42 out of 51 eligible operators (82%). Final plan contains 3 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 38 out of 47 eligible operators (80%). Final plan contains 3 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q54.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q54.native_datafusion/extended.txt index 9e1691c2fc..af2b39b224 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q54.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q54.native_datafusion/extended.txt @@ -35,11 +35,7 @@ CometNativeColumnarToRow : : : : : : : +- CometProject : : : : : : : +- CometFilter : : : : : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : : : : : +- CometSubqueryBroadcast - : : : : : : : +- CometBroadcastExchange - : : : : : : : +- CometProject - : : : : : : : +- CometFilter - : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : : +- ReusedSubquery : : : : : : +- CometBroadcastExchange : : : : : : +- CometProject : : : : : : +- CometFilter @@ -104,4 +100,4 @@ CometNativeColumnarToRow : +- CometNativeScan parquet spark_catalog.default.date_dim +- CometNativeScan parquet spark_catalog.default.date_dim -Comet accelerated 93 out of 100 eligible operators (93%). Final plan contains 5 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 89 out of 96 eligible operators (92%). Final plan contains 5 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q56.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q56.native_datafusion/extended.txt index 0744bc307d..e8b3e8e97f 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q56.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q56.native_datafusion/extended.txt @@ -48,11 +48,7 @@ CometNativeColumnarToRow : : : +- CometBroadcastHashJoin : : : :- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : +- CometSubqueryBroadcast - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- ReusedSubquery : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter @@ -81,11 +77,7 @@ CometNativeColumnarToRow : : +- CometBroadcastHashJoin : : :- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : +- CometSubqueryBroadcast - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- ReusedSubquery : : +- CometBroadcastExchange : : +- CometProject : : +- CometFilter @@ -104,4 +96,4 @@ CometNativeColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.item -Comet accelerated 101 out of 104 eligible operators (97%). Final plan contains 1 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 93 out of 96 eligible operators (96%). Final plan contains 1 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q58.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q58.native_datafusion/extended.txt index ed81d13b0e..93bffeba01 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q58.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q58.native_datafusion/extended.txt @@ -67,11 +67,7 @@ CometNativeColumnarToRow : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter - : : : : +- Subquery - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- ReusedSubquery : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- CometBroadcastExchange : : +- CometProject @@ -98,21 +94,7 @@ CometNativeColumnarToRow : +- CometBroadcastHashJoin : :- CometFilter : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : +- CometSubqueryBroadcast - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : : +- Subquery - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- ReusedSubquery : +- CometBroadcastExchange : +- CometProject : +- CometFilter @@ -128,4 +110,4 @@ CometNativeColumnarToRow : +- ReusedSubquery +- CometNativeScan parquet spark_catalog.default.date_dim -Comet accelerated 115 out of 124 eligible operators (92%). Final plan contains 5 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 100 out of 108 eligible operators (92%). Final plan contains 3 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q60.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q60.native_datafusion/extended.txt index 0744bc307d..e8b3e8e97f 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q60.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q60.native_datafusion/extended.txt @@ -48,11 +48,7 @@ CometNativeColumnarToRow : : : +- CometBroadcastHashJoin : : : :- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : +- CometSubqueryBroadcast - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- ReusedSubquery : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter @@ -81,11 +77,7 @@ CometNativeColumnarToRow : : +- CometBroadcastHashJoin : : :- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : +- CometSubqueryBroadcast - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- ReusedSubquery : : +- CometBroadcastExchange : : +- CometProject : : +- CometFilter @@ -104,4 +96,4 @@ CometNativeColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.item -Comet accelerated 101 out of 104 eligible operators (97%). Final plan contains 1 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 93 out of 96 eligible operators (96%). Final plan contains 1 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q61.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q61.native_datafusion/extended.txt index 8d91cbe7a2..39f985df28 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q61.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q61.native_datafusion/extended.txt @@ -63,11 +63,7 @@ Project : : : : +- CometBroadcastHashJoin : : : : :- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : : : +- CometSubqueryBroadcast - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- ReusedSubquery : : : : +- CometBroadcastExchange : : : : +- CometProject : : : : +- CometFilter @@ -88,4 +84,4 @@ Project +- CometFilter +- CometNativeScan parquet spark_catalog.default.item -Comet accelerated 82 out of 87 eligible operators (94%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 78 out of 83 eligible operators (93%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q65.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q65.native_datafusion/extended.txt index 21d60497a1..b3937f6c97 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q65.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q65.native_datafusion/extended.txt @@ -42,14 +42,10 @@ CometNativeColumnarToRow +- CometBroadcastHashJoin :- CometFilter : +- CometNativeScan parquet spark_catalog.default.store_sales - : +- CometSubqueryBroadcast - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- ReusedSubquery +- CometBroadcastExchange +- CometProject +- CometFilter +- CometNativeScan parquet spark_catalog.default.date_dim -Comet accelerated 50 out of 52 eligible operators (96%). Final plan contains 1 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 46 out of 48 eligible operators (95%). Final plan contains 1 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q66.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q66.native_datafusion/extended.txt index c07ecfe46a..3b8d1937f2 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q66.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q66.native_datafusion/extended.txt @@ -49,10 +49,7 @@ CometNativeColumnarToRow : : : +- CometBroadcastHashJoin : : : :- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : +- CometSubqueryBroadcast - : : : : +- CometBroadcastExchange - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- ReusedSubquery : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter @@ -69,4 +66,4 @@ CometNativeColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.ship_mode -Comet accelerated 67 out of 69 eligible operators (97%). Final plan contains 1 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 64 out of 66 eligible operators (96%). Final plan contains 1 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q69.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q69.native_datafusion/extended.txt index 6d398580fb..cb518a5c47 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q69.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q69.native_datafusion/extended.txt @@ -32,11 +32,7 @@ TakeOrderedAndProject : : : +- CometProject : : : +- CometBroadcastHashJoin : : : :- CometNativeScan parquet spark_catalog.default.web_sales - : : : : +- CometSubqueryBroadcast - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- ReusedSubquery : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter @@ -46,11 +42,7 @@ TakeOrderedAndProject : : +- CometProject : : +- CometBroadcastHashJoin : : :- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : +- CometSubqueryBroadcast - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- ReusedSubquery : : +- CometBroadcastExchange : : +- CometProject : : +- CometFilter @@ -66,4 +58,4 @@ TakeOrderedAndProject +- CometFilter +- CometNativeScan parquet spark_catalog.default.customer_demographics -Comet accelerated 44 out of 61 eligible operators (72%). Final plan contains 6 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 36 out of 53 eligible operators (67%). Final plan contains 6 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q70.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q70.native_datafusion/extended.txt index 7655338234..33b51b7046 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q70.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q70.native_datafusion/extended.txt @@ -46,11 +46,7 @@ TakeOrderedAndProject : +- CometBroadcastHashJoin : :- CometFilter : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : +- CometSubqueryBroadcast - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- ReusedSubquery : +- CometBroadcastExchange : +- CometProject : +- CometFilter @@ -60,4 +56,4 @@ TakeOrderedAndProject +- CometFilter +- CometNativeScan parquet spark_catalog.default.date_dim -Comet accelerated 39 out of 56 eligible operators (69%). Final plan contains 5 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 35 out of 52 eligible operators (67%). Final plan contains 5 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q71.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q71.native_datafusion/extended.txt index 834098d4ee..2c056d01d7 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q71.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q71.native_datafusion/extended.txt @@ -30,11 +30,7 @@ CometNativeColumnarToRow : : +- CometBroadcastHashJoin : : :- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : +- CometSubqueryBroadcast - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- ReusedSubquery : : +- CometBroadcastExchange : : +- CometProject : : +- CometFilter @@ -43,11 +39,7 @@ CometNativeColumnarToRow : +- CometBroadcastHashJoin : :- CometFilter : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : +- CometSubqueryBroadcast - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- ReusedSubquery : +- CometBroadcastExchange : +- CometProject : +- CometFilter @@ -57,4 +49,4 @@ CometNativeColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.time_dim -Comet accelerated 54 out of 57 eligible operators (94%). Final plan contains 1 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 46 out of 49 eligible operators (93%). Final plan contains 1 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q74.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q74.native_datafusion/extended.txt index 7768138645..cfe634568f 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q74.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q74.native_datafusion/extended.txt @@ -62,10 +62,7 @@ CometNativeColumnarToRow : : +- CometBroadcastExchange : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : +- CometSubqueryBroadcast - : : +- CometBroadcastExchange - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- ReusedSubquery : +- CometBroadcastExchange : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -83,12 +80,9 @@ CometNativeColumnarToRow : +- CometBroadcastExchange : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.web_sales - : +- CometSubqueryBroadcast - : +- CometBroadcastExchange - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- ReusedSubquery +- CometBroadcastExchange +- CometFilter +- CometNativeScan parquet spark_catalog.default.date_dim -Comet accelerated 87 out of 91 eligible operators (95%). Final plan contains 1 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 81 out of 85 eligible operators (95%). Final plan contains 1 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q75.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q75.native_datafusion/extended.txt index ee7c0af216..bccd23b141 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q75.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q75.native_datafusion/extended.txt @@ -48,10 +48,7 @@ CometNativeColumnarToRow : : : : +- CometBroadcastHashJoin : : : : :- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : : : +- CometSubqueryBroadcast - : : : : : +- CometBroadcastExchange - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- ReusedSubquery : : : : +- CometBroadcastExchange : : : : +- CometProject : : : : +- CometFilter @@ -74,10 +71,7 @@ CometNativeColumnarToRow : : : +- CometBroadcastHashJoin : : : :- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : : +- CometSubqueryBroadcast - : : : : +- CometBroadcastExchange - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- ReusedSubquery : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter @@ -136,10 +130,7 @@ CometNativeColumnarToRow : : : +- CometBroadcastHashJoin : : : :- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : : +- CometSubqueryBroadcast - : : : : +- CometBroadcastExchange - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- ReusedSubquery : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter @@ -162,10 +153,7 @@ CometNativeColumnarToRow : : +- CometBroadcastHashJoin : : :- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : +- CometSubqueryBroadcast - : : : +- CometBroadcastExchange - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- ReusedSubquery : : +- CometBroadcastExchange : : +- CometProject : : +- CometFilter @@ -179,4 +167,4 @@ CometNativeColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.web_returns -Comet accelerated 173 out of 179 eligible operators (96%). Final plan contains 1 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 161 out of 167 eligible operators (96%). Final plan contains 1 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q77.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q77.native_datafusion/extended.txt index 8805a2e6c2..38ee51e753 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q77.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q77.native_datafusion/extended.txt @@ -39,11 +39,7 @@ TakeOrderedAndProject : : +- CometBroadcastHashJoin : : :- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.store_returns - : : : +- CometSubqueryBroadcast - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- ReusedSubquery : : +- CometBroadcastExchange : : +- CometProject : : +- CometFilter @@ -61,11 +57,7 @@ TakeOrderedAndProject : : +- CometProject : : +- CometBroadcastHashJoin : : :- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : +- CometSubqueryBroadcast - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- ReusedSubquery : : +- CometBroadcastExchange : : +- CometProject : : +- CometFilter @@ -77,11 +69,7 @@ TakeOrderedAndProject : +- CometProject : +- CometBroadcastHashJoin : :- CometNativeScan parquet spark_catalog.default.catalog_returns - : : +- CometSubqueryBroadcast - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- ReusedSubquery : +- CometBroadcastExchange : +- CometProject : +- CometFilter @@ -98,11 +86,7 @@ TakeOrderedAndProject : : +- CometBroadcastHashJoin : : :- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : +- CometSubqueryBroadcast - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- ReusedSubquery : : +- CometBroadcastExchange : : +- CometProject : : +- CometFilter @@ -120,11 +104,7 @@ TakeOrderedAndProject : +- CometBroadcastHashJoin : :- CometFilter : : +- CometNativeScan parquet spark_catalog.default.web_returns - : : +- CometSubqueryBroadcast - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- ReusedSubquery : +- CometBroadcastExchange : +- CometProject : +- CometFilter @@ -133,4 +113,4 @@ TakeOrderedAndProject +- CometFilter +- CometNativeScan parquet spark_catalog.default.web_page -Comet accelerated 115 out of 129 eligible operators (89%). Final plan contains 5 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 95 out of 109 eligible operators (87%). Final plan contains 5 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q78.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q78.native_datafusion/extended.txt index 8426b9653c..d29dbc13e5 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q78.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q78.native_datafusion/extended.txt @@ -43,10 +43,7 @@ TakeOrderedAndProject : : : +- CometExchange : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : +- CometSubqueryBroadcast - : : : +- CometBroadcastExchange - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- ReusedSubquery : : +- CometSort : : +- CometExchange : : +- CometProject @@ -69,10 +66,7 @@ TakeOrderedAndProject : : +- CometExchange : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : +- CometSubqueryBroadcast - : : +- CometBroadcastExchange - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- ReusedSubquery : +- CometSort : +- CometExchange : +- CometProject @@ -82,4 +76,4 @@ TakeOrderedAndProject +- CometFilter +- CometNativeScan parquet spark_catalog.default.date_dim -Comet accelerated 77 out of 82 eligible operators (93%). Final plan contains 1 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 71 out of 76 eligible operators (93%). Final plan contains 1 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q80.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q80.native_datafusion/extended.txt index 27e32a1602..9f67d9b8b5 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q80.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q80.native_datafusion/extended.txt @@ -65,11 +65,7 @@ CometNativeColumnarToRow : : : : : : +- CometExchange : : : : : : +- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : : : +- CometSubqueryBroadcast - : : : : : : +- CometBroadcastExchange - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : +- ReusedSubquery : : : : : +- CometSort : : : : : +- CometExchange : : : : : +- CometProject @@ -108,11 +104,7 @@ CometNativeColumnarToRow : : : : : +- CometExchange : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : : : +- CometSubqueryBroadcast - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- ReusedSubquery : : : : +- CometSort : : : : +- CometExchange : : : : +- CometProject @@ -135,4 +127,4 @@ CometNativeColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.promotion -Comet accelerated 132 out of 135 eligible operators (97%). Final plan contains 1 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 124 out of 127 eligible operators (97%). Final plan contains 1 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q81.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q81.native_datafusion/extended.txt index 6d9aa9697d..f10339641b 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q81.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q81.native_datafusion/extended.txt @@ -43,11 +43,7 @@ CometNativeColumnarToRow : : : +- CometBroadcastHashJoin : : : :- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.catalog_returns - : : : : +- CometSubqueryBroadcast - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- ReusedSubquery : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter @@ -65,4 +61,4 @@ CometNativeColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.customer_address -Comet accelerated 63 out of 65 eligible operators (96%). Final plan contains 1 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 59 out of 61 eligible operators (96%). Final plan contains 1 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q83.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q83.native_datafusion/extended.txt index e4d1992492..ef8debea9a 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q83.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q83.native_datafusion/extended.txt @@ -95,20 +95,7 @@ CometNativeColumnarToRow : +- CometBroadcastHashJoin : :- CometFilter : : +- CometNativeScan parquet spark_catalog.default.web_returns - : : +- CometSubqueryBroadcast - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometNativeScan parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- ReusedSubquery : +- CometBroadcastExchange : +- CometProject : +- CometFilter @@ -127,4 +114,4 @@ CometNativeColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.date_dim -Comet accelerated 124 out of 127 eligible operators (97%). Final plan contains 1 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 111 out of 114 eligible operators (97%). Final plan contains 1 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q87.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q87.native_datafusion/extended.txt index 5c80165c1f..7680e50295 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q87.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q87.native_datafusion/extended.txt @@ -39,11 +39,7 @@ HashAggregate : : +- CometBroadcastHashJoin : : :- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : +- CometSubqueryBroadcast - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- ReusedSubquery : : +- CometBroadcastExchange : : +- CometProject : : +- CometFilter @@ -63,11 +59,7 @@ HashAggregate : +- CometBroadcastHashJoin : :- CometFilter : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : +- CometSubqueryBroadcast - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- ReusedSubquery : +- CometBroadcastExchange : +- CometProject : +- CometFilter @@ -77,4 +69,4 @@ HashAggregate +- CometFilter +- CometNativeScan parquet spark_catalog.default.customer -Comet accelerated 64 out of 74 eligible operators (86%). Final plan contains 4 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 56 out of 66 eligible operators (84%). Final plan contains 4 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q92.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q92.native_datafusion/extended.txt index 40926fafac..bcf7db282b 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q92.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q92.native_datafusion/extended.txt @@ -28,11 +28,7 @@ CometNativeColumnarToRow : +- CometBroadcastHashJoin : :- CometFilter : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : +- CometSubqueryBroadcast - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- ReusedSubquery : +- CometBroadcastExchange : +- CometProject : +- CometFilter @@ -42,4 +38,4 @@ CometNativeColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.date_dim -Comet accelerated 40 out of 42 eligible operators (95%). Final plan contains 1 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 36 out of 38 eligible operators (94%). Final plan contains 1 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q97.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q97.native_datafusion/extended.txt index 74163dff2b..a56bd059fa 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q97.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q97.native_datafusion/extended.txt @@ -27,14 +27,10 @@ CometNativeColumnarToRow +- CometProject +- CometBroadcastHashJoin :- CometNativeScan parquet spark_catalog.default.catalog_sales - : +- CometSubqueryBroadcast - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- ReusedSubquery +- CometBroadcastExchange +- CometProject +- CometFilter +- CometNativeScan parquet spark_catalog.default.date_dim -Comet accelerated 35 out of 37 eligible operators (94%). Final plan contains 1 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 31 out of 33 eligible operators (93%). Final plan contains 1 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q10a.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q10a.native_datafusion/extended.txt index 173c450c71..f27de2fdbb 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q10a.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q10a.native_datafusion/extended.txt @@ -30,11 +30,7 @@ CometNativeColumnarToRow : : :- CometProject : : : +- CometBroadcastHashJoin : : : :- CometNativeScan parquet spark_catalog.default.web_sales - : : : : +- CometSubqueryBroadcast - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- ReusedSubquery : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter @@ -42,11 +38,7 @@ CometNativeColumnarToRow : : +- CometProject : : +- CometBroadcastHashJoin : : :- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : +- CometSubqueryBroadcast - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- ReusedSubquery : : +- CometBroadcastExchange : : +- CometProject : : +- CometFilter @@ -60,4 +52,4 @@ CometNativeColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.customer_demographics -Comet accelerated 57 out of 60 eligible operators (95%). Final plan contains 1 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 49 out of 52 eligible operators (94%). Final plan contains 1 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q11.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q11.native_datafusion/extended.txt index 7768138645..cfe634568f 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q11.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q11.native_datafusion/extended.txt @@ -62,10 +62,7 @@ CometNativeColumnarToRow : : +- CometBroadcastExchange : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : +- CometSubqueryBroadcast - : : +- CometBroadcastExchange - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- ReusedSubquery : +- CometBroadcastExchange : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -83,12 +80,9 @@ CometNativeColumnarToRow : +- CometBroadcastExchange : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.web_sales - : +- CometSubqueryBroadcast - : +- CometBroadcastExchange - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- ReusedSubquery +- CometBroadcastExchange +- CometFilter +- CometNativeScan parquet spark_catalog.default.date_dim -Comet accelerated 87 out of 91 eligible operators (95%). Final plan contains 1 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 81 out of 85 eligible operators (95%). Final plan contains 1 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q14.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q14.native_datafusion/extended.txt index 2d20c40e4b..1b9b46662f 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q14.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q14.native_datafusion/extended.txt @@ -11,11 +11,7 @@ CometNativeColumnarToRow : : :- CometProject : : : +- CometBroadcastHashJoin : : : :- CometNativeScan parquet spark_catalog.default.store_sales - : : : : +- CometSubqueryBroadcast - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- ReusedSubquery : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter @@ -23,11 +19,7 @@ CometNativeColumnarToRow : : :- CometProject : : : +- CometBroadcastHashJoin : : : :- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : +- CometSubqueryBroadcast - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- ReusedSubquery : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter @@ -35,11 +27,7 @@ CometNativeColumnarToRow : : +- CometProject : : +- CometBroadcastHashJoin : : :- CometNativeScan parquet spark_catalog.default.web_sales - : : : +- CometSubqueryBroadcast - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- ReusedSubquery : : +- CometBroadcastExchange : : +- CometProject : : +- CometFilter @@ -96,11 +84,7 @@ CometNativeColumnarToRow : : : : : : +- CometBroadcastHashJoin : : : : : : :- CometFilter : : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : : : : +- CometSubqueryBroadcast - : : : : : : : +- CometBroadcastExchange - : : : : : : : +- CometProject - : : : : : : : +- CometFilter - : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : : +- ReusedSubquery : : : : : : +- CometBroadcastExchange : : : : : : +- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.item @@ -119,11 +103,7 @@ CometNativeColumnarToRow : : : : +- CometBroadcastHashJoin : : : : :- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : : : +- CometSubqueryBroadcast - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- ReusedSubquery : : : : +- CometBroadcastExchange : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.item @@ -167,11 +147,7 @@ CometNativeColumnarToRow : : : : : +- CometBroadcastHashJoin : : : : : :- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : : : +- CometSubqueryBroadcast - : : : : : : +- CometBroadcastExchange - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : +- ReusedSubquery : : : : : +- CometBroadcastExchange : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.item @@ -190,11 +166,7 @@ CometNativeColumnarToRow : : : +- CometBroadcastHashJoin : : : :- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : : +- CometSubqueryBroadcast - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- ReusedSubquery : : : +- CometBroadcastExchange : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.item @@ -266,11 +238,7 @@ CometNativeColumnarToRow : : : : : +- CometBroadcastHashJoin : : : : : :- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : : : +- CometSubqueryBroadcast - : : : : : : +- CometBroadcastExchange - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : +- ReusedSubquery : : : : : +- CometBroadcastExchange : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.item @@ -289,11 +257,7 @@ CometNativeColumnarToRow : : : +- CometBroadcastHashJoin : : : :- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : : +- CometSubqueryBroadcast - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- ReusedSubquery : : : +- CometBroadcastExchange : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.item @@ -337,11 +301,7 @@ CometNativeColumnarToRow : : : : +- CometBroadcastHashJoin : : : : :- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : : +- CometSubqueryBroadcast - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- ReusedSubquery : : : : +- CometBroadcastExchange : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.item @@ -360,11 +320,7 @@ CometNativeColumnarToRow : : +- CometBroadcastHashJoin : : :- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : +- CometSubqueryBroadcast - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- ReusedSubquery : : +- CometBroadcastExchange : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.item @@ -382,4 +338,4 @@ CometNativeColumnarToRow : +- CometNativeScan parquet spark_catalog.default.date_dim +- CometNativeScan parquet spark_catalog.default.date_dim -Comet accelerated 354 out of 377 eligible operators (93%). Final plan contains 6 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 310 out of 333 eligible operators (93%). Final plan contains 6 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q14a.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q14a.native_datafusion/extended.txt index 83b545c34c..64d8fff5be 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q14a.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q14a.native_datafusion/extended.txt @@ -18,11 +18,7 @@ CometNativeColumnarToRow : : : :- CometProject : : : : +- CometBroadcastHashJoin : : : : :- CometNativeScan parquet spark_catalog.default.store_sales - : : : : : +- CometSubqueryBroadcast - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- ReusedSubquery : : : : +- CometBroadcastExchange : : : : +- CometProject : : : : +- CometFilter @@ -42,11 +38,7 @@ CometNativeColumnarToRow : : : +- CometProject : : : +- CometBroadcastHashJoin : : : :- CometNativeScan parquet spark_catalog.default.web_sales - : : : : +- CometSubqueryBroadcast - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- ReusedSubquery : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter @@ -98,11 +90,7 @@ CometNativeColumnarToRow : : : : : : : +- CometBroadcastHashJoin : : : : : : : :- CometFilter : : : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : : : : : +- CometSubqueryBroadcast - : : : : : : : : +- CometBroadcastExchange - : : : : : : : : +- CometProject - : : : : : : : : +- CometFilter - : : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : : : +- ReusedSubquery : : : : : : : +- CometBroadcastExchange : : : : : : : +- CometFilter : : : : : : : +- CometNativeScan parquet spark_catalog.default.item @@ -121,11 +109,7 @@ CometNativeColumnarToRow : : : : : +- CometBroadcastHashJoin : : : : : :- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : : : : +- CometSubqueryBroadcast - : : : : : : +- CometBroadcastExchange - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : +- ReusedSubquery : : : : : +- CometBroadcastExchange : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.item @@ -169,11 +153,7 @@ CometNativeColumnarToRow : : : : : : +- CometBroadcastHashJoin : : : : : : :- CometFilter : : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : : : : +- CometSubqueryBroadcast - : : : : : : : +- CometBroadcastExchange - : : : : : : : +- CometProject - : : : : : : : +- CometFilter - : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : : +- ReusedSubquery : : : : : : +- CometBroadcastExchange : : : : : : +- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.item @@ -192,11 +172,7 @@ CometNativeColumnarToRow : : : : +- CometBroadcastHashJoin : : : : :- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : : : +- CometSubqueryBroadcast - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- ReusedSubquery : : : : +- CometBroadcastExchange : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.item @@ -220,11 +196,7 @@ CometNativeColumnarToRow : : : :- CometBroadcastHashJoin : : : : :- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : : +- CometSubqueryBroadcast - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- ReusedSubquery : : : : +- CometBroadcastExchange : : : : +- CometProject : : : : +- CometBroadcastHashJoin @@ -257,11 +229,7 @@ CometNativeColumnarToRow : : : : : : : +- CometBroadcastHashJoin : : : : : : : :- CometFilter : : : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : : : : : +- CometSubqueryBroadcast - : : : : : : : : +- CometBroadcastExchange - : : : : : : : : +- CometProject - : : : : : : : : +- CometFilter - : : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : : : +- ReusedSubquery : : : : : : : +- CometBroadcastExchange : : : : : : : +- CometFilter : : : : : : : +- CometNativeScan parquet spark_catalog.default.item @@ -280,11 +248,7 @@ CometNativeColumnarToRow : : : : : +- CometBroadcastHashJoin : : : : : :- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : : : : +- CometSubqueryBroadcast - : : : : : : +- CometBroadcastExchange - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : +- ReusedSubquery : : : : : +- CometBroadcastExchange : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.item @@ -328,11 +292,7 @@ CometNativeColumnarToRow : : : : : : +- CometBroadcastHashJoin : : : : : : :- CometFilter : : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : : : : +- CometSubqueryBroadcast - : : : : : : : +- CometBroadcastExchange - : : : : : : : +- CometProject - : : : : : : : +- CometFilter - : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : : +- ReusedSubquery : : : : : : +- CometBroadcastExchange : : : : : : +- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.item @@ -351,11 +311,7 @@ CometNativeColumnarToRow : : : : +- CometBroadcastHashJoin : : : : :- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : : : +- CometSubqueryBroadcast - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- ReusedSubquery : : : : +- CometBroadcastExchange : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.item @@ -379,11 +335,7 @@ CometNativeColumnarToRow : : :- CometBroadcastHashJoin : : : :- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : : +- CometSubqueryBroadcast - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- ReusedSubquery : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometBroadcastHashJoin @@ -416,11 +368,7 @@ CometNativeColumnarToRow : : : : : : +- CometBroadcastHashJoin : : : : : : :- CometFilter : : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : : : : +- CometSubqueryBroadcast - : : : : : : : +- CometBroadcastExchange - : : : : : : : +- CometProject - : : : : : : : +- CometFilter - : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : : +- ReusedSubquery : : : : : : +- CometBroadcastExchange : : : : : : +- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.item @@ -439,11 +387,7 @@ CometNativeColumnarToRow : : : : +- CometBroadcastHashJoin : : : : :- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : : : +- CometSubqueryBroadcast - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- ReusedSubquery : : : : +- CometBroadcastExchange : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.item @@ -487,11 +431,7 @@ CometNativeColumnarToRow : : : : : +- CometBroadcastHashJoin : : : : : :- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : : : +- CometSubqueryBroadcast - : : : : : : +- CometBroadcastExchange - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : +- ReusedSubquery : : : : : +- CometBroadcastExchange : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.item @@ -510,11 +450,7 @@ CometNativeColumnarToRow : : : +- CometBroadcastHashJoin : : : :- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : : +- CometSubqueryBroadcast - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- ReusedSubquery : : : +- CometBroadcastExchange : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.item @@ -543,11 +479,7 @@ CometNativeColumnarToRow : : : :- CometProject : : : : +- CometBroadcastHashJoin : : : : :- CometNativeScan parquet spark_catalog.default.store_sales - : : : : : +- CometSubqueryBroadcast - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- ReusedSubquery : : : : +- CometBroadcastExchange : : : : +- CometProject : : : : +- CometFilter @@ -567,11 +499,7 @@ CometNativeColumnarToRow : : : +- CometProject : : : +- CometBroadcastHashJoin : : : :- CometNativeScan parquet spark_catalog.default.web_sales - : : : : +- CometSubqueryBroadcast - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- ReusedSubquery : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter @@ -623,11 +551,7 @@ CometNativeColumnarToRow : : : : : : : +- CometBroadcastHashJoin : : : : : : : :- CometFilter : : : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : : : : : +- CometSubqueryBroadcast - : : : : : : : : +- CometBroadcastExchange - : : : : : : : : +- CometProject - : : : : : : : : +- CometFilter - : : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : : : +- ReusedSubquery : : : : : : : +- CometBroadcastExchange : : : : : : : +- CometFilter : : : : : : : +- CometNativeScan parquet spark_catalog.default.item @@ -646,11 +570,7 @@ CometNativeColumnarToRow : : : : : +- CometBroadcastHashJoin : : : : : :- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : : : : +- CometSubqueryBroadcast - : : : : : : +- CometBroadcastExchange - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : +- ReusedSubquery : : : : : +- CometBroadcastExchange : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.item @@ -694,11 +614,7 @@ CometNativeColumnarToRow : : : : : : +- CometBroadcastHashJoin : : : : : : :- CometFilter : : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : : : : +- CometSubqueryBroadcast - : : : : : : : +- CometBroadcastExchange - : : : : : : : +- CometProject - : : : : : : : +- CometFilter - : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : : +- ReusedSubquery : : : : : : +- CometBroadcastExchange : : : : : : +- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.item @@ -717,11 +633,7 @@ CometNativeColumnarToRow : : : : +- CometBroadcastHashJoin : : : : :- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : : : +- CometSubqueryBroadcast - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- ReusedSubquery : : : : +- CometBroadcastExchange : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.item @@ -745,11 +657,7 @@ CometNativeColumnarToRow : : : :- CometBroadcastHashJoin : : : : :- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : : +- CometSubqueryBroadcast - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- ReusedSubquery : : : : +- CometBroadcastExchange : : : : +- CometProject : : : : +- CometBroadcastHashJoin @@ -782,11 +690,7 @@ CometNativeColumnarToRow : : : : : : : +- CometBroadcastHashJoin : : : : : : : :- CometFilter : : : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : : : : : +- CometSubqueryBroadcast - : : : : : : : : +- CometBroadcastExchange - : : : : : : : : +- CometProject - : : : : : : : : +- CometFilter - : : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : : : +- ReusedSubquery : : : : : : : +- CometBroadcastExchange : : : : : : : +- CometFilter : : : : : : : +- CometNativeScan parquet spark_catalog.default.item @@ -805,11 +709,7 @@ CometNativeColumnarToRow : : : : : +- CometBroadcastHashJoin : : : : : :- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : : : : +- CometSubqueryBroadcast - : : : : : : +- CometBroadcastExchange - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : +- ReusedSubquery : : : : : +- CometBroadcastExchange : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.item @@ -853,11 +753,7 @@ CometNativeColumnarToRow : : : : : : +- CometBroadcastHashJoin : : : : : : :- CometFilter : : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : : : : +- CometSubqueryBroadcast - : : : : : : : +- CometBroadcastExchange - : : : : : : : +- CometProject - : : : : : : : +- CometFilter - : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : : +- ReusedSubquery : : : : : : +- CometBroadcastExchange : : : : : : +- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.item @@ -876,11 +772,7 @@ CometNativeColumnarToRow : : : : +- CometBroadcastHashJoin : : : : :- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : : : +- CometSubqueryBroadcast - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- ReusedSubquery : : : : +- CometBroadcastExchange : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.item @@ -904,11 +796,7 @@ CometNativeColumnarToRow : : :- CometBroadcastHashJoin : : : :- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : : +- CometSubqueryBroadcast - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- ReusedSubquery : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometBroadcastHashJoin @@ -941,11 +829,7 @@ CometNativeColumnarToRow : : : : : : +- CometBroadcastHashJoin : : : : : : :- CometFilter : : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : : : : +- CometSubqueryBroadcast - : : : : : : : +- CometBroadcastExchange - : : : : : : : +- CometProject - : : : : : : : +- CometFilter - : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : : +- ReusedSubquery : : : : : : +- CometBroadcastExchange : : : : : : +- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.item @@ -964,11 +848,7 @@ CometNativeColumnarToRow : : : : +- CometBroadcastHashJoin : : : : :- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : : : +- CometSubqueryBroadcast - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- ReusedSubquery : : : : +- CometBroadcastExchange : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.item @@ -1012,11 +892,7 @@ CometNativeColumnarToRow : : : : : +- CometBroadcastHashJoin : : : : : :- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : : : +- CometSubqueryBroadcast - : : : : : : +- CometBroadcastExchange - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : +- ReusedSubquery : : : : : +- CometBroadcastExchange : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.item @@ -1035,11 +911,7 @@ CometNativeColumnarToRow : : : +- CometBroadcastHashJoin : : : :- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : : +- CometSubqueryBroadcast - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- ReusedSubquery : : : +- CometBroadcastExchange : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.item @@ -1068,11 +940,7 @@ CometNativeColumnarToRow : : : :- CometProject : : : : +- CometBroadcastHashJoin : : : : :- CometNativeScan parquet spark_catalog.default.store_sales - : : : : : +- CometSubqueryBroadcast - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- ReusedSubquery : : : : +- CometBroadcastExchange : : : : +- CometProject : : : : +- CometFilter @@ -1092,11 +960,7 @@ CometNativeColumnarToRow : : : +- CometProject : : : +- CometBroadcastHashJoin : : : :- CometNativeScan parquet spark_catalog.default.web_sales - : : : : +- CometSubqueryBroadcast - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- ReusedSubquery : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter @@ -1148,11 +1012,7 @@ CometNativeColumnarToRow : : : : : : : +- CometBroadcastHashJoin : : : : : : : :- CometFilter : : : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : : : : : +- CometSubqueryBroadcast - : : : : : : : : +- CometBroadcastExchange - : : : : : : : : +- CometProject - : : : : : : : : +- CometFilter - : : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : : : +- ReusedSubquery : : : : : : : +- CometBroadcastExchange : : : : : : : +- CometFilter : : : : : : : +- CometNativeScan parquet spark_catalog.default.item @@ -1171,11 +1031,7 @@ CometNativeColumnarToRow : : : : : +- CometBroadcastHashJoin : : : : : :- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : : : : +- CometSubqueryBroadcast - : : : : : : +- CometBroadcastExchange - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : +- ReusedSubquery : : : : : +- CometBroadcastExchange : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.item @@ -1219,11 +1075,7 @@ CometNativeColumnarToRow : : : : : : +- CometBroadcastHashJoin : : : : : : :- CometFilter : : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : : : : +- CometSubqueryBroadcast - : : : : : : : +- CometBroadcastExchange - : : : : : : : +- CometProject - : : : : : : : +- CometFilter - : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : : +- ReusedSubquery : : : : : : +- CometBroadcastExchange : : : : : : +- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.item @@ -1242,11 +1094,7 @@ CometNativeColumnarToRow : : : : +- CometBroadcastHashJoin : : : : :- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : : : +- CometSubqueryBroadcast - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- ReusedSubquery : : : : +- CometBroadcastExchange : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.item @@ -1270,11 +1118,7 @@ CometNativeColumnarToRow : : : :- CometBroadcastHashJoin : : : : :- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : : +- CometSubqueryBroadcast - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- ReusedSubquery : : : : +- CometBroadcastExchange : : : : +- CometProject : : : : +- CometBroadcastHashJoin @@ -1307,11 +1151,7 @@ CometNativeColumnarToRow : : : : : : : +- CometBroadcastHashJoin : : : : : : : :- CometFilter : : : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : : : : : +- CometSubqueryBroadcast - : : : : : : : : +- CometBroadcastExchange - : : : : : : : : +- CometProject - : : : : : : : : +- CometFilter - : : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : : : +- ReusedSubquery : : : : : : : +- CometBroadcastExchange : : : : : : : +- CometFilter : : : : : : : +- CometNativeScan parquet spark_catalog.default.item @@ -1330,11 +1170,7 @@ CometNativeColumnarToRow : : : : : +- CometBroadcastHashJoin : : : : : :- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : : : : +- CometSubqueryBroadcast - : : : : : : +- CometBroadcastExchange - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : +- ReusedSubquery : : : : : +- CometBroadcastExchange : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.item @@ -1378,11 +1214,7 @@ CometNativeColumnarToRow : : : : : : +- CometBroadcastHashJoin : : : : : : :- CometFilter : : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : : : : +- CometSubqueryBroadcast - : : : : : : : +- CometBroadcastExchange - : : : : : : : +- CometProject - : : : : : : : +- CometFilter - : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : : +- ReusedSubquery : : : : : : +- CometBroadcastExchange : : : : : : +- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.item @@ -1401,11 +1233,7 @@ CometNativeColumnarToRow : : : : +- CometBroadcastHashJoin : : : : :- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : : : +- CometSubqueryBroadcast - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- ReusedSubquery : : : : +- CometBroadcastExchange : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.item @@ -1429,11 +1257,7 @@ CometNativeColumnarToRow : : :- CometBroadcastHashJoin : : : :- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : : +- CometSubqueryBroadcast - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- ReusedSubquery : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometBroadcastHashJoin @@ -1466,11 +1290,7 @@ CometNativeColumnarToRow : : : : : : +- CometBroadcastHashJoin : : : : : : :- CometFilter : : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : : : : +- CometSubqueryBroadcast - : : : : : : : +- CometBroadcastExchange - : : : : : : : +- CometProject - : : : : : : : +- CometFilter - : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : : +- ReusedSubquery : : : : : : +- CometBroadcastExchange : : : : : : +- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.item @@ -1489,11 +1309,7 @@ CometNativeColumnarToRow : : : : +- CometBroadcastHashJoin : : : : :- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : : : +- CometSubqueryBroadcast - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- ReusedSubquery : : : : +- CometBroadcastExchange : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.item @@ -1537,11 +1353,7 @@ CometNativeColumnarToRow : : : : : +- CometBroadcastHashJoin : : : : : :- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : : : +- CometSubqueryBroadcast - : : : : : : +- CometBroadcastExchange - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : +- ReusedSubquery : : : : : +- CometBroadcastExchange : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.item @@ -1560,11 +1372,7 @@ CometNativeColumnarToRow : : : +- CometBroadcastHashJoin : : : :- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : : +- CometSubqueryBroadcast - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- ReusedSubquery : : : +- CometBroadcastExchange : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.item @@ -1593,11 +1401,7 @@ CometNativeColumnarToRow : : : :- CometProject : : : : +- CometBroadcastHashJoin : : : : :- CometNativeScan parquet spark_catalog.default.store_sales - : : : : : +- CometSubqueryBroadcast - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- ReusedSubquery : : : : +- CometBroadcastExchange : : : : +- CometProject : : : : +- CometFilter @@ -1617,11 +1421,7 @@ CometNativeColumnarToRow : : : +- CometProject : : : +- CometBroadcastHashJoin : : : :- CometNativeScan parquet spark_catalog.default.web_sales - : : : : +- CometSubqueryBroadcast - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- ReusedSubquery : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter @@ -1673,11 +1473,7 @@ CometNativeColumnarToRow : : : : : : : +- CometBroadcastHashJoin : : : : : : : :- CometFilter : : : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : : : : : +- CometSubqueryBroadcast - : : : : : : : : +- CometBroadcastExchange - : : : : : : : : +- CometProject - : : : : : : : : +- CometFilter - : : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : : : +- ReusedSubquery : : : : : : : +- CometBroadcastExchange : : : : : : : +- CometFilter : : : : : : : +- CometNativeScan parquet spark_catalog.default.item @@ -1696,11 +1492,7 @@ CometNativeColumnarToRow : : : : : +- CometBroadcastHashJoin : : : : : :- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : : : : +- CometSubqueryBroadcast - : : : : : : +- CometBroadcastExchange - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : +- ReusedSubquery : : : : : +- CometBroadcastExchange : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.item @@ -1744,11 +1536,7 @@ CometNativeColumnarToRow : : : : : : +- CometBroadcastHashJoin : : : : : : :- CometFilter : : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : : : : +- CometSubqueryBroadcast - : : : : : : : +- CometBroadcastExchange - : : : : : : : +- CometProject - : : : : : : : +- CometFilter - : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : : +- ReusedSubquery : : : : : : +- CometBroadcastExchange : : : : : : +- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.item @@ -1767,11 +1555,7 @@ CometNativeColumnarToRow : : : : +- CometBroadcastHashJoin : : : : :- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : : : +- CometSubqueryBroadcast - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- ReusedSubquery : : : : +- CometBroadcastExchange : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.item @@ -1795,11 +1579,7 @@ CometNativeColumnarToRow : : : :- CometBroadcastHashJoin : : : : :- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : : +- CometSubqueryBroadcast - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- ReusedSubquery : : : : +- CometBroadcastExchange : : : : +- CometProject : : : : +- CometBroadcastHashJoin @@ -1832,11 +1612,7 @@ CometNativeColumnarToRow : : : : : : : +- CometBroadcastHashJoin : : : : : : : :- CometFilter : : : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : : : : : +- CometSubqueryBroadcast - : : : : : : : : +- CometBroadcastExchange - : : : : : : : : +- CometProject - : : : : : : : : +- CometFilter - : : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : : : +- ReusedSubquery : : : : : : : +- CometBroadcastExchange : : : : : : : +- CometFilter : : : : : : : +- CometNativeScan parquet spark_catalog.default.item @@ -1855,11 +1631,7 @@ CometNativeColumnarToRow : : : : : +- CometBroadcastHashJoin : : : : : :- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : : : : +- CometSubqueryBroadcast - : : : : : : +- CometBroadcastExchange - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : +- ReusedSubquery : : : : : +- CometBroadcastExchange : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.item @@ -1903,11 +1675,7 @@ CometNativeColumnarToRow : : : : : : +- CometBroadcastHashJoin : : : : : : :- CometFilter : : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : : : : +- CometSubqueryBroadcast - : : : : : : : +- CometBroadcastExchange - : : : : : : : +- CometProject - : : : : : : : +- CometFilter - : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : : +- ReusedSubquery : : : : : : +- CometBroadcastExchange : : : : : : +- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.item @@ -1926,11 +1694,7 @@ CometNativeColumnarToRow : : : : +- CometBroadcastHashJoin : : : : :- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : : : +- CometSubqueryBroadcast - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- ReusedSubquery : : : : +- CometBroadcastExchange : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.item @@ -1954,11 +1718,7 @@ CometNativeColumnarToRow : : :- CometBroadcastHashJoin : : : :- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : : +- CometSubqueryBroadcast - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- ReusedSubquery : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometBroadcastHashJoin @@ -1991,11 +1751,7 @@ CometNativeColumnarToRow : : : : : : +- CometBroadcastHashJoin : : : : : : :- CometFilter : : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : : : : +- CometSubqueryBroadcast - : : : : : : : +- CometBroadcastExchange - : : : : : : : +- CometProject - : : : : : : : +- CometFilter - : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : : +- ReusedSubquery : : : : : : +- CometBroadcastExchange : : : : : : +- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.item @@ -2014,11 +1770,7 @@ CometNativeColumnarToRow : : : : +- CometBroadcastHashJoin : : : : :- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : : : +- CometSubqueryBroadcast - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- ReusedSubquery : : : : +- CometBroadcastExchange : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.item @@ -2062,11 +1814,7 @@ CometNativeColumnarToRow : : : : : +- CometBroadcastHashJoin : : : : : :- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : : : +- CometSubqueryBroadcast - : : : : : : +- CometBroadcastExchange - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : +- ReusedSubquery : : : : : +- CometBroadcastExchange : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.item @@ -2085,11 +1833,7 @@ CometNativeColumnarToRow : : : +- CometBroadcastHashJoin : : : :- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : : +- CometSubqueryBroadcast - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- ReusedSubquery : : : +- CometBroadcastExchange : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.item @@ -2118,11 +1862,7 @@ CometNativeColumnarToRow : : :- CometProject : : : +- CometBroadcastHashJoin : : : :- CometNativeScan parquet spark_catalog.default.store_sales - : : : : +- CometSubqueryBroadcast - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- ReusedSubquery : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter @@ -2142,11 +1882,7 @@ CometNativeColumnarToRow : : +- CometProject : : +- CometBroadcastHashJoin : : :- CometNativeScan parquet spark_catalog.default.web_sales - : : : +- CometSubqueryBroadcast - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- ReusedSubquery : : +- CometBroadcastExchange : : +- CometProject : : +- CometFilter @@ -2198,11 +1934,7 @@ CometNativeColumnarToRow : : : : : : +- CometBroadcastHashJoin : : : : : : :- CometFilter : : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : : : : +- CometSubqueryBroadcast - : : : : : : : +- CometBroadcastExchange - : : : : : : : +- CometProject - : : : : : : : +- CometFilter - : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : : +- ReusedSubquery : : : : : : +- CometBroadcastExchange : : : : : : +- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.item @@ -2221,11 +1953,7 @@ CometNativeColumnarToRow : : : : +- CometBroadcastHashJoin : : : : :- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : : : +- CometSubqueryBroadcast - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- ReusedSubquery : : : : +- CometBroadcastExchange : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.item @@ -2269,11 +1997,7 @@ CometNativeColumnarToRow : : : : : +- CometBroadcastHashJoin : : : : : :- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : : : +- CometSubqueryBroadcast - : : : : : : +- CometBroadcastExchange - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : +- ReusedSubquery : : : : : +- CometBroadcastExchange : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.item @@ -2292,11 +2016,7 @@ CometNativeColumnarToRow : : : +- CometBroadcastHashJoin : : : :- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : : +- CometSubqueryBroadcast - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- ReusedSubquery : : : +- CometBroadcastExchange : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.item @@ -2320,11 +2040,7 @@ CometNativeColumnarToRow : : :- CometBroadcastHashJoin : : : :- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : +- CometSubqueryBroadcast - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- ReusedSubquery : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometBroadcastHashJoin @@ -2357,11 +2073,7 @@ CometNativeColumnarToRow : : : : : : +- CometBroadcastHashJoin : : : : : : :- CometFilter : : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : : : : +- CometSubqueryBroadcast - : : : : : : : +- CometBroadcastExchange - : : : : : : : +- CometProject - : : : : : : : +- CometFilter - : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : : +- ReusedSubquery : : : : : : +- CometBroadcastExchange : : : : : : +- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.item @@ -2380,11 +2092,7 @@ CometNativeColumnarToRow : : : : +- CometBroadcastHashJoin : : : : :- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : : : +- CometSubqueryBroadcast - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- ReusedSubquery : : : : +- CometBroadcastExchange : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.item @@ -2428,11 +2136,7 @@ CometNativeColumnarToRow : : : : : +- CometBroadcastHashJoin : : : : : :- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : : : +- CometSubqueryBroadcast - : : : : : : +- CometBroadcastExchange - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : +- ReusedSubquery : : : : : +- CometBroadcastExchange : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.item @@ -2451,11 +2155,7 @@ CometNativeColumnarToRow : : : +- CometBroadcastHashJoin : : : :- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : : +- CometSubqueryBroadcast - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- ReusedSubquery : : : +- CometBroadcastExchange : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.item @@ -2479,11 +2179,7 @@ CometNativeColumnarToRow : :- CometBroadcastHashJoin : : :- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : +- CometSubqueryBroadcast - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- ReusedSubquery : : +- CometBroadcastExchange : : +- CometProject : : +- CometBroadcastHashJoin @@ -2516,11 +2212,7 @@ CometNativeColumnarToRow : : : : : +- CometBroadcastHashJoin : : : : : :- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : : : +- CometSubqueryBroadcast - : : : : : : +- CometBroadcastExchange - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : +- ReusedSubquery : : : : : +- CometBroadcastExchange : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.item @@ -2539,11 +2231,7 @@ CometNativeColumnarToRow : : : +- CometBroadcastHashJoin : : : :- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : : +- CometSubqueryBroadcast - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- ReusedSubquery : : : +- CometBroadcastExchange : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.item @@ -2587,11 +2275,7 @@ CometNativeColumnarToRow : : : : +- CometBroadcastHashJoin : : : : :- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : : +- CometSubqueryBroadcast - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- ReusedSubquery : : : : +- CometBroadcastExchange : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.item @@ -2610,11 +2294,7 @@ CometNativeColumnarToRow : : +- CometBroadcastHashJoin : : :- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : +- CometSubqueryBroadcast - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- ReusedSubquery : : +- CometBroadcastExchange : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.item @@ -2627,4 +2307,4 @@ CometNativeColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.date_dim -Comet accelerated 2487 out of 2622 eligible operators (94%). Final plan contains 6 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 2167 out of 2302 eligible operators (94%). Final plan contains 6 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q18a.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q18a.native_datafusion/extended.txt index b34ebfdf84..75f132f6fb 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q18a.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q18a.native_datafusion/extended.txt @@ -63,11 +63,7 @@ CometNativeColumnarToRow : : : : : : +- CometBroadcastHashJoin : : : : : : :- CometFilter : : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : : : : +- CometSubqueryBroadcast - : : : : : : : +- CometBroadcastExchange - : : : : : : : +- CometProject - : : : : : : : +- CometFilter - : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : : +- ReusedSubquery : : : : : : +- CometBroadcastExchange : : : : : : +- CometProject : : : : : : +- CometFilter @@ -108,11 +104,7 @@ CometNativeColumnarToRow : : : : : : +- CometBroadcastHashJoin : : : : : : :- CometFilter : : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : : : : +- CometSubqueryBroadcast - : : : : : : : +- CometBroadcastExchange - : : : : : : : +- CometProject - : : : : : : : +- CometFilter - : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : : +- ReusedSubquery : : : : : : +- CometBroadcastExchange : : : : : : +- CometProject : : : : : : +- CometFilter @@ -153,11 +145,7 @@ CometNativeColumnarToRow : : : : : : +- CometBroadcastHashJoin : : : : : : :- CometFilter : : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : : : : +- CometSubqueryBroadcast - : : : : : : : +- CometBroadcastExchange - : : : : : : : +- CometProject - : : : : : : : +- CometFilter - : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : : +- ReusedSubquery : : : : : : +- CometBroadcastExchange : : : : : : +- CometProject : : : : : : +- CometFilter @@ -198,11 +186,7 @@ CometNativeColumnarToRow : : : : : +- CometBroadcastHashJoin : : : : : :- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : : : +- CometSubqueryBroadcast - : : : : : : +- CometBroadcastExchange - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : +- ReusedSubquery : : : : : +- CometBroadcastExchange : : : : : +- CometProject : : : : : +- CometFilter @@ -226,4 +210,4 @@ CometNativeColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.item -Comet accelerated 221 out of 226 eligible operators (97%). Final plan contains 1 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 205 out of 210 eligible operators (97%). Final plan contains 1 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q27a.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q27a.native_datafusion/extended.txt index 9c4497cbd3..70614792ba 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q27a.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q27a.native_datafusion/extended.txt @@ -48,11 +48,7 @@ CometNativeColumnarToRow : : : : +- CometBroadcastHashJoin : : : : :- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : : : +- CometSubqueryBroadcast - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- ReusedSubquery : : : : +- CometBroadcastExchange : : : : +- CometProject : : : : +- CometFilter @@ -82,11 +78,7 @@ CometNativeColumnarToRow : : : +- CometBroadcastHashJoin : : : :- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : : +- CometSubqueryBroadcast - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- ReusedSubquery : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter @@ -103,4 +95,4 @@ CometNativeColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.item -Comet accelerated 100 out of 103 eligible operators (97%). Final plan contains 1 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 92 out of 95 eligible operators (96%). Final plan contains 1 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q35.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q35.native_datafusion/extended.txt index 2e79f6f464..b7c8209556 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q35.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q35.native_datafusion/extended.txt @@ -33,11 +33,7 @@ TakeOrderedAndProject : : : +- CometProject : : : +- CometBroadcastHashJoin : : : :- CometNativeScan parquet spark_catalog.default.web_sales - : : : : +- CometSubqueryBroadcast - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- ReusedSubquery : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter @@ -47,11 +43,7 @@ TakeOrderedAndProject : : +- CometProject : : +- CometBroadcastHashJoin : : :- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : +- CometSubqueryBroadcast - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- ReusedSubquery : : +- CometBroadcastExchange : : +- CometProject : : +- CometFilter @@ -67,4 +59,4 @@ TakeOrderedAndProject +- CometFilter +- CometNativeScan parquet spark_catalog.default.customer_demographics -Comet accelerated 44 out of 62 eligible operators (70%). Final plan contains 6 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 36 out of 54 eligible operators (66%). Final plan contains 6 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q35a.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q35a.native_datafusion/extended.txt index 173c450c71..f27de2fdbb 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q35a.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q35a.native_datafusion/extended.txt @@ -30,11 +30,7 @@ CometNativeColumnarToRow : : :- CometProject : : : +- CometBroadcastHashJoin : : : :- CometNativeScan parquet spark_catalog.default.web_sales - : : : : +- CometSubqueryBroadcast - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- ReusedSubquery : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter @@ -42,11 +38,7 @@ CometNativeColumnarToRow : : +- CometProject : : +- CometBroadcastHashJoin : : :- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : +- CometSubqueryBroadcast - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- ReusedSubquery : : +- CometBroadcastExchange : : +- CometProject : : +- CometFilter @@ -60,4 +52,4 @@ CometNativeColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.customer_demographics -Comet accelerated 57 out of 60 eligible operators (95%). Final plan contains 1 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 49 out of 52 eligible operators (94%). Final plan contains 1 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q49.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q49.native_datafusion/extended.txt index f2ae44fb86..cd6c39e4f3 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q49.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q49.native_datafusion/extended.txt @@ -54,11 +54,7 @@ CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : +- CometSubqueryBroadcast - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- ReusedSubquery : : +- CometProject : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.catalog_returns @@ -85,11 +81,7 @@ CometNativeColumnarToRow : : +- CometProject : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : +- CometSubqueryBroadcast - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- ReusedSubquery : +- CometProject : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.store_returns @@ -98,4 +90,4 @@ CometNativeColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.date_dim -Comet accelerated 75 out of 95 eligible operators (78%). Final plan contains 4 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 67 out of 87 eligible operators (77%). Final plan contains 4 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q51a.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q51a.native_datafusion/extended.txt index a3400c5ecf..c8010c1286 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q51a.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q51a.native_datafusion/extended.txt @@ -82,11 +82,7 @@ TakeOrderedAndProject : : +- CometBroadcastHashJoin : : :- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : +- CometSubqueryBroadcast - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- ReusedSubquery : : +- CometBroadcastExchange : : +- CometProject : : +- CometFilter @@ -104,11 +100,7 @@ TakeOrderedAndProject : +- CometBroadcastHashJoin : :- CometFilter : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : +- CometSubqueryBroadcast - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- ReusedSubquery : +- CometBroadcastExchange : +- CometProject : +- CometFilter @@ -193,11 +185,7 @@ TakeOrderedAndProject : +- CometBroadcastHashJoin : :- CometFilter : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : +- CometSubqueryBroadcast - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- ReusedSubquery : +- CometBroadcastExchange : +- CometProject : +- CometFilter @@ -215,14 +203,10 @@ TakeOrderedAndProject +- CometBroadcastHashJoin :- CometFilter : +- CometNativeScan parquet spark_catalog.default.store_sales - : +- CometSubqueryBroadcast - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- ReusedSubquery +- CometBroadcastExchange +- CometProject +- CometFilter +- CometNativeScan parquet spark_catalog.default.date_dim -Comet accelerated 158 out of 212 eligible operators (74%). Final plan contains 14 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 142 out of 196 eligible operators (72%). Final plan contains 14 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q5a.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q5a.native_datafusion/extended.txt index b472530c10..8d61d1e1ad 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q5a.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q5a.native_datafusion/extended.txt @@ -27,11 +27,7 @@ CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.store_returns - : : : : +- CometSubqueryBroadcast - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- ReusedSubquery : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter @@ -51,19 +47,11 @@ CometNativeColumnarToRow : : : : :- CometProject : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : : +- CometSubqueryBroadcast - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- ReusedSubquery : : : : +- CometProject : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.catalog_returns - : : : : +- CometSubqueryBroadcast - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- ReusedSubquery : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter @@ -83,20 +71,12 @@ CometNativeColumnarToRow : : : :- CometProject : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : : +- CometSubqueryBroadcast - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- ReusedSubquery : : : +- CometProject : : : +- CometBroadcastHashJoin : : : :- CometBroadcastExchange : : : : +- CometNativeScan parquet spark_catalog.default.web_returns - : : : : +- CometSubqueryBroadcast - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- ReusedSubquery : : : +- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.web_sales @@ -134,11 +114,7 @@ CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.store_returns - : : : : +- CometSubqueryBroadcast - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- ReusedSubquery : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter @@ -158,19 +134,11 @@ CometNativeColumnarToRow : : : : :- CometProject : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : : +- CometSubqueryBroadcast - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- ReusedSubquery : : : : +- CometProject : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.catalog_returns - : : : : +- CometSubqueryBroadcast - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- ReusedSubquery : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter @@ -190,20 +158,12 @@ CometNativeColumnarToRow : : : :- CometProject : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : : +- CometSubqueryBroadcast - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- ReusedSubquery : : : +- CometProject : : : +- CometBroadcastHashJoin : : : :- CometBroadcastExchange : : : : +- CometNativeScan parquet spark_catalog.default.web_returns - : : : : +- CometSubqueryBroadcast - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- ReusedSubquery : : : +- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.web_sales @@ -241,11 +201,7 @@ CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.store_returns - : : : +- CometSubqueryBroadcast - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- ReusedSubquery : : +- CometBroadcastExchange : : +- CometProject : : +- CometFilter @@ -265,19 +221,11 @@ CometNativeColumnarToRow : : : :- CometProject : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : +- CometSubqueryBroadcast - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- ReusedSubquery : : : +- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.catalog_returns - : : : +- CometSubqueryBroadcast - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- ReusedSubquery : : +- CometBroadcastExchange : : +- CometProject : : +- CometFilter @@ -297,20 +245,12 @@ CometNativeColumnarToRow : : :- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : +- CometSubqueryBroadcast - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- ReusedSubquery : : +- CometProject : : +- CometBroadcastHashJoin : : :- CometBroadcastExchange : : : +- CometNativeScan parquet spark_catalog.default.web_returns - : : : +- CometSubqueryBroadcast - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- ReusedSubquery : : +- CometProject : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.web_sales @@ -323,4 +263,4 @@ CometNativeColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.web_site -Comet accelerated 305 out of 323 eligible operators (94%). Final plan contains 1 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 245 out of 263 eligible operators (93%). Final plan contains 1 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q70a.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q70a.native_datafusion/extended.txt index 97dd294f61..a60ad5ebc5 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q70a.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q70a.native_datafusion/extended.txt @@ -50,11 +50,7 @@ TakeOrderedAndProject : : +- CometBroadcastHashJoin : : :- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : +- CometSubqueryBroadcast - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- ReusedSubquery : : +- CometBroadcastExchange : : +- CometProject : : +- CometFilter @@ -109,11 +105,7 @@ TakeOrderedAndProject : : +- CometBroadcastHashJoin : : :- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : +- CometSubqueryBroadcast - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- ReusedSubquery : : +- CometBroadcastExchange : : +- CometProject : : +- CometFilter @@ -168,11 +160,7 @@ TakeOrderedAndProject : +- CometBroadcastHashJoin : :- CometFilter : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : +- CometSubqueryBroadcast - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- ReusedSubquery : +- CometBroadcastExchange : +- CometProject : +- CometFilter @@ -182,4 +170,4 @@ TakeOrderedAndProject +- CometFilter +- CometNativeScan parquet spark_catalog.default.date_dim -Comet accelerated 117 out of 168 eligible operators (69%). Final plan contains 15 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 105 out of 156 eligible operators (67%). Final plan contains 15 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q74.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q74.native_datafusion/extended.txt index 7768138645..cfe634568f 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q74.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q74.native_datafusion/extended.txt @@ -62,10 +62,7 @@ CometNativeColumnarToRow : : +- CometBroadcastExchange : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : +- CometSubqueryBroadcast - : : +- CometBroadcastExchange - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- ReusedSubquery : +- CometBroadcastExchange : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -83,12 +80,9 @@ CometNativeColumnarToRow : +- CometBroadcastExchange : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.web_sales - : +- CometSubqueryBroadcast - : +- CometBroadcastExchange - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- ReusedSubquery +- CometBroadcastExchange +- CometFilter +- CometNativeScan parquet spark_catalog.default.date_dim -Comet accelerated 87 out of 91 eligible operators (95%). Final plan contains 1 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 81 out of 85 eligible operators (95%). Final plan contains 1 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q75.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q75.native_datafusion/extended.txt index ee7c0af216..bccd23b141 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q75.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q75.native_datafusion/extended.txt @@ -48,10 +48,7 @@ CometNativeColumnarToRow : : : : +- CometBroadcastHashJoin : : : : :- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : : : +- CometSubqueryBroadcast - : : : : : +- CometBroadcastExchange - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- ReusedSubquery : : : : +- CometBroadcastExchange : : : : +- CometProject : : : : +- CometFilter @@ -74,10 +71,7 @@ CometNativeColumnarToRow : : : +- CometBroadcastHashJoin : : : :- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : : +- CometSubqueryBroadcast - : : : : +- CometBroadcastExchange - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- ReusedSubquery : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter @@ -136,10 +130,7 @@ CometNativeColumnarToRow : : : +- CometBroadcastHashJoin : : : :- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : : +- CometSubqueryBroadcast - : : : : +- CometBroadcastExchange - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- ReusedSubquery : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter @@ -162,10 +153,7 @@ CometNativeColumnarToRow : : +- CometBroadcastHashJoin : : :- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : +- CometSubqueryBroadcast - : : : +- CometBroadcastExchange - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- ReusedSubquery : : +- CometBroadcastExchange : : +- CometProject : : +- CometFilter @@ -179,4 +167,4 @@ CometNativeColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.web_returns -Comet accelerated 173 out of 179 eligible operators (96%). Final plan contains 1 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 161 out of 167 eligible operators (96%). Final plan contains 1 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q77a.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q77a.native_datafusion/extended.txt index 6621cfd8f8..95b315d258 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q77a.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q77a.native_datafusion/extended.txt @@ -43,11 +43,7 @@ CometNativeColumnarToRow : : : +- CometBroadcastHashJoin : : : :- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.store_returns - : : : : +- CometSubqueryBroadcast - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- ReusedSubquery : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter @@ -65,11 +61,7 @@ CometNativeColumnarToRow : : : +- CometProject : : : +- CometBroadcastHashJoin : : : :- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : +- CometSubqueryBroadcast - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- ReusedSubquery : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter @@ -81,11 +73,7 @@ CometNativeColumnarToRow : : +- CometProject : : +- CometBroadcastHashJoin : : :- CometNativeScan parquet spark_catalog.default.catalog_returns - : : : +- CometSubqueryBroadcast - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- ReusedSubquery : : +- CometBroadcastExchange : : +- CometProject : : +- CometFilter @@ -102,11 +90,7 @@ CometNativeColumnarToRow : : : +- CometBroadcastHashJoin : : : :- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : : +- CometSubqueryBroadcast - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- ReusedSubquery : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter @@ -124,11 +108,7 @@ CometNativeColumnarToRow : : +- CometBroadcastHashJoin : : :- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.web_returns - : : : +- CometSubqueryBroadcast - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- ReusedSubquery : : +- CometBroadcastExchange : : +- CometProject : : +- CometFilter @@ -179,11 +159,7 @@ CometNativeColumnarToRow : : : +- CometBroadcastHashJoin : : : :- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.store_returns - : : : : +- CometSubqueryBroadcast - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- ReusedSubquery : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter @@ -201,11 +177,7 @@ CometNativeColumnarToRow : : : +- CometProject : : : +- CometBroadcastHashJoin : : : :- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : +- CometSubqueryBroadcast - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- ReusedSubquery : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter @@ -217,11 +189,7 @@ CometNativeColumnarToRow : : +- CometProject : : +- CometBroadcastHashJoin : : :- CometNativeScan parquet spark_catalog.default.catalog_returns - : : : +- CometSubqueryBroadcast - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- ReusedSubquery : : +- CometBroadcastExchange : : +- CometProject : : +- CometFilter @@ -238,11 +206,7 @@ CometNativeColumnarToRow : : : +- CometBroadcastHashJoin : : : :- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : : +- CometSubqueryBroadcast - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- ReusedSubquery : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter @@ -260,11 +224,7 @@ CometNativeColumnarToRow : : +- CometBroadcastHashJoin : : :- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.web_returns - : : : +- CometSubqueryBroadcast - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- ReusedSubquery : : +- CometBroadcastExchange : : +- CometProject : : +- CometFilter @@ -315,11 +275,7 @@ CometNativeColumnarToRow : : +- CometBroadcastHashJoin : : :- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.store_returns - : : : +- CometSubqueryBroadcast - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- ReusedSubquery : : +- CometBroadcastExchange : : +- CometProject : : +- CometFilter @@ -337,11 +293,7 @@ CometNativeColumnarToRow : : +- CometProject : : +- CometBroadcastHashJoin : : :- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : +- CometSubqueryBroadcast - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- ReusedSubquery : : +- CometBroadcastExchange : : +- CometProject : : +- CometFilter @@ -353,11 +305,7 @@ CometNativeColumnarToRow : +- CometProject : +- CometBroadcastHashJoin : :- CometNativeScan parquet spark_catalog.default.catalog_returns - : : +- CometSubqueryBroadcast - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- ReusedSubquery : +- CometBroadcastExchange : +- CometProject : +- CometFilter @@ -374,11 +322,7 @@ CometNativeColumnarToRow : : +- CometBroadcastHashJoin : : :- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : +- CometSubqueryBroadcast - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- ReusedSubquery : : +- CometBroadcastExchange : : +- CometProject : : +- CometFilter @@ -396,11 +340,7 @@ CometNativeColumnarToRow : +- CometBroadcastHashJoin : :- CometFilter : : +- CometNativeScan parquet spark_catalog.default.web_returns - : : +- CometSubqueryBroadcast - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- ReusedSubquery : +- CometBroadcastExchange : +- CometProject : +- CometFilter @@ -409,4 +349,4 @@ CometNativeColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.web_page -Comet accelerated 350 out of 392 eligible operators (89%). Final plan contains 18 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 290 out of 332 eligible operators (87%). Final plan contains 18 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q78.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q78.native_datafusion/extended.txt index 8426b9653c..d29dbc13e5 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q78.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q78.native_datafusion/extended.txt @@ -43,10 +43,7 @@ TakeOrderedAndProject : : : +- CometExchange : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : +- CometSubqueryBroadcast - : : : +- CometBroadcastExchange - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- ReusedSubquery : : +- CometSort : : +- CometExchange : : +- CometProject @@ -69,10 +66,7 @@ TakeOrderedAndProject : : +- CometExchange : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : +- CometSubqueryBroadcast - : : +- CometBroadcastExchange - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- ReusedSubquery : +- CometSort : +- CometExchange : +- CometProject @@ -82,4 +76,4 @@ TakeOrderedAndProject +- CometFilter +- CometNativeScan parquet spark_catalog.default.date_dim -Comet accelerated 77 out of 82 eligible operators (93%). Final plan contains 1 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 71 out of 76 eligible operators (93%). Final plan contains 1 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q80a.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q80a.native_datafusion/extended.txt index dced05333a..10e0803953 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q80a.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q80a.native_datafusion/extended.txt @@ -68,11 +68,7 @@ CometNativeColumnarToRow : : : : : : : +- CometExchange : : : : : : : +- CometFilter : : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : : : : +- CometSubqueryBroadcast - : : : : : : : +- CometBroadcastExchange - : : : : : : : +- CometProject - : : : : : : : +- CometFilter - : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : : +- ReusedSubquery : : : : : : +- CometSort : : : : : : +- CometExchange : : : : : : +- CometProject @@ -111,11 +107,7 @@ CometNativeColumnarToRow : : : : : : +- CometExchange : : : : : : +- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : : : : +- CometSubqueryBroadcast - : : : : : : +- CometBroadcastExchange - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : +- ReusedSubquery : : : : : +- CometSort : : : : : +- CometExchange : : : : : +- CometProject @@ -204,11 +196,7 @@ CometNativeColumnarToRow : : : : : : : +- CometExchange : : : : : : : +- CometFilter : : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : : : : +- CometSubqueryBroadcast - : : : : : : : +- CometBroadcastExchange - : : : : : : : +- CometProject - : : : : : : : +- CometFilter - : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : : +- ReusedSubquery : : : : : : +- CometSort : : : : : : +- CometExchange : : : : : : +- CometProject @@ -247,11 +235,7 @@ CometNativeColumnarToRow : : : : : : +- CometExchange : : : : : : +- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : : : : +- CometSubqueryBroadcast - : : : : : : +- CometBroadcastExchange - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : +- ReusedSubquery : : : : : +- CometSort : : : : : +- CometExchange : : : : : +- CometProject @@ -340,11 +324,7 @@ CometNativeColumnarToRow : : : : : : +- CometExchange : : : : : : +- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : : : +- CometSubqueryBroadcast - : : : : : : +- CometBroadcastExchange - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : +- ReusedSubquery : : : : : +- CometSort : : : : : +- CometExchange : : : : : +- CometProject @@ -383,11 +363,7 @@ CometNativeColumnarToRow : : : : : +- CometExchange : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : : : +- CometSubqueryBroadcast - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- ReusedSubquery : : : : +- CometSort : : : : +- CometExchange : : : : +- CometProject @@ -410,4 +386,4 @@ CometNativeColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.promotion -Comet accelerated 401 out of 410 eligible operators (97%). Final plan contains 1 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 377 out of 386 eligible operators (97%). Final plan contains 1 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q10a.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q10a.native_datafusion/extended.txt index 173c450c71..f27de2fdbb 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q10a.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q10a.native_datafusion/extended.txt @@ -30,11 +30,7 @@ CometNativeColumnarToRow : : :- CometProject : : : +- CometBroadcastHashJoin : : : :- CometNativeScan parquet spark_catalog.default.web_sales - : : : : +- CometSubqueryBroadcast - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- ReusedSubquery : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter @@ -42,11 +38,7 @@ CometNativeColumnarToRow : : +- CometProject : : +- CometBroadcastHashJoin : : :- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : +- CometSubqueryBroadcast - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- ReusedSubquery : : +- CometBroadcastExchange : : +- CometProject : : +- CometFilter @@ -60,4 +52,4 @@ CometNativeColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.customer_demographics -Comet accelerated 57 out of 60 eligible operators (95%). Final plan contains 1 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 49 out of 52 eligible operators (94%). Final plan contains 1 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q11.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q11.native_datafusion/extended.txt index 7768138645..cfe634568f 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q11.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q11.native_datafusion/extended.txt @@ -62,10 +62,7 @@ CometNativeColumnarToRow : : +- CometBroadcastExchange : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : +- CometSubqueryBroadcast - : : +- CometBroadcastExchange - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- ReusedSubquery : +- CometBroadcastExchange : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -83,12 +80,9 @@ CometNativeColumnarToRow : +- CometBroadcastExchange : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.web_sales - : +- CometSubqueryBroadcast - : +- CometBroadcastExchange - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- ReusedSubquery +- CometBroadcastExchange +- CometFilter +- CometNativeScan parquet spark_catalog.default.date_dim -Comet accelerated 87 out of 91 eligible operators (95%). Final plan contains 1 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 81 out of 85 eligible operators (95%). Final plan contains 1 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q14.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q14.native_datafusion/extended.txt index 341ae1fb04..85cda8b553 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q14.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q14.native_datafusion/extended.txt @@ -11,11 +11,7 @@ CometNativeColumnarToRow : : :- CometProject : : : +- CometBroadcastHashJoin : : : :- CometNativeScan parquet spark_catalog.default.store_sales - : : : : +- CometSubqueryBroadcast - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- ReusedSubquery : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter @@ -23,11 +19,7 @@ CometNativeColumnarToRow : : :- CometProject : : : +- CometBroadcastHashJoin : : : :- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : +- CometSubqueryBroadcast - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- ReusedSubquery : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter @@ -35,11 +27,7 @@ CometNativeColumnarToRow : : +- CometProject : : +- CometBroadcastHashJoin : : :- CometNativeScan parquet spark_catalog.default.web_sales - : : : +- CometSubqueryBroadcast - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- ReusedSubquery : : +- CometBroadcastExchange : : +- CometProject : : +- CometFilter @@ -58,11 +46,7 @@ CometNativeColumnarToRow : : : : +- CometBroadcastExchange : : : : +- CometProject : : : : +- CometFilter - : : : : : +- Subquery - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- ReusedSubquery : : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : : +- Subquery : : : : +- CometNativeColumnarToRow @@ -101,11 +85,7 @@ CometNativeColumnarToRow : : : : : : +- CometBroadcastHashJoin : : : : : : :- CometFilter : : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : : : : +- CometSubqueryBroadcast - : : : : : : : +- CometBroadcastExchange - : : : : : : : +- CometProject - : : : : : : : +- CometFilter - : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : : +- ReusedSubquery : : : : : : +- CometBroadcastExchange : : : : : : +- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.item @@ -124,11 +104,7 @@ CometNativeColumnarToRow : : : : +- CometBroadcastHashJoin : : : : :- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : : : +- CometSubqueryBroadcast - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- ReusedSubquery : : : : +- CometBroadcastExchange : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.item @@ -172,11 +148,7 @@ CometNativeColumnarToRow : : : : : +- CometBroadcastHashJoin : : : : : :- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : : : +- CometSubqueryBroadcast - : : : : : : +- CometBroadcastExchange - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : +- ReusedSubquery : : : : : +- CometBroadcastExchange : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.item @@ -195,11 +167,7 @@ CometNativeColumnarToRow : : : +- CometBroadcastHashJoin : : : :- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : : +- CometSubqueryBroadcast - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- ReusedSubquery : : : +- CometBroadcastExchange : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.item @@ -234,11 +202,7 @@ CometNativeColumnarToRow : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter - : : : : +- Subquery - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- ReusedSubquery : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : : +- Subquery : : : +- CometNativeColumnarToRow @@ -277,11 +241,7 @@ CometNativeColumnarToRow : : : : : +- CometBroadcastHashJoin : : : : : :- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : : : +- CometSubqueryBroadcast - : : : : : : +- CometBroadcastExchange - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : +- ReusedSubquery : : : : : +- CometBroadcastExchange : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.item @@ -300,11 +260,7 @@ CometNativeColumnarToRow : : : +- CometBroadcastHashJoin : : : :- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : : +- CometSubqueryBroadcast - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- ReusedSubquery : : : +- CometBroadcastExchange : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.item @@ -348,11 +304,7 @@ CometNativeColumnarToRow : : : : +- CometBroadcastHashJoin : : : : :- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : : +- CometSubqueryBroadcast - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- ReusedSubquery : : : : +- CometBroadcastExchange : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.item @@ -371,11 +323,7 @@ CometNativeColumnarToRow : : +- CometBroadcastHashJoin : : :- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : +- CometSubqueryBroadcast - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- ReusedSubquery : : +- CometBroadcastExchange : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.item @@ -394,4 +342,4 @@ CometNativeColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.date_dim -Comet accelerated 360 out of 387 eligible operators (93%). Final plan contains 8 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 310 out of 337 eligible operators (91%). Final plan contains 6 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q14a.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q14a.native_datafusion/extended.txt index 83b545c34c..64d8fff5be 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q14a.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q14a.native_datafusion/extended.txt @@ -18,11 +18,7 @@ CometNativeColumnarToRow : : : :- CometProject : : : : +- CometBroadcastHashJoin : : : : :- CometNativeScan parquet spark_catalog.default.store_sales - : : : : : +- CometSubqueryBroadcast - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- ReusedSubquery : : : : +- CometBroadcastExchange : : : : +- CometProject : : : : +- CometFilter @@ -42,11 +38,7 @@ CometNativeColumnarToRow : : : +- CometProject : : : +- CometBroadcastHashJoin : : : :- CometNativeScan parquet spark_catalog.default.web_sales - : : : : +- CometSubqueryBroadcast - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- ReusedSubquery : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter @@ -98,11 +90,7 @@ CometNativeColumnarToRow : : : : : : : +- CometBroadcastHashJoin : : : : : : : :- CometFilter : : : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : : : : : +- CometSubqueryBroadcast - : : : : : : : : +- CometBroadcastExchange - : : : : : : : : +- CometProject - : : : : : : : : +- CometFilter - : : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : : : +- ReusedSubquery : : : : : : : +- CometBroadcastExchange : : : : : : : +- CometFilter : : : : : : : +- CometNativeScan parquet spark_catalog.default.item @@ -121,11 +109,7 @@ CometNativeColumnarToRow : : : : : +- CometBroadcastHashJoin : : : : : :- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : : : : +- CometSubqueryBroadcast - : : : : : : +- CometBroadcastExchange - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : +- ReusedSubquery : : : : : +- CometBroadcastExchange : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.item @@ -169,11 +153,7 @@ CometNativeColumnarToRow : : : : : : +- CometBroadcastHashJoin : : : : : : :- CometFilter : : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : : : : +- CometSubqueryBroadcast - : : : : : : : +- CometBroadcastExchange - : : : : : : : +- CometProject - : : : : : : : +- CometFilter - : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : : +- ReusedSubquery : : : : : : +- CometBroadcastExchange : : : : : : +- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.item @@ -192,11 +172,7 @@ CometNativeColumnarToRow : : : : +- CometBroadcastHashJoin : : : : :- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : : : +- CometSubqueryBroadcast - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- ReusedSubquery : : : : +- CometBroadcastExchange : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.item @@ -220,11 +196,7 @@ CometNativeColumnarToRow : : : :- CometBroadcastHashJoin : : : : :- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : : +- CometSubqueryBroadcast - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- ReusedSubquery : : : : +- CometBroadcastExchange : : : : +- CometProject : : : : +- CometBroadcastHashJoin @@ -257,11 +229,7 @@ CometNativeColumnarToRow : : : : : : : +- CometBroadcastHashJoin : : : : : : : :- CometFilter : : : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : : : : : +- CometSubqueryBroadcast - : : : : : : : : +- CometBroadcastExchange - : : : : : : : : +- CometProject - : : : : : : : : +- CometFilter - : : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : : : +- ReusedSubquery : : : : : : : +- CometBroadcastExchange : : : : : : : +- CometFilter : : : : : : : +- CometNativeScan parquet spark_catalog.default.item @@ -280,11 +248,7 @@ CometNativeColumnarToRow : : : : : +- CometBroadcastHashJoin : : : : : :- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : : : : +- CometSubqueryBroadcast - : : : : : : +- CometBroadcastExchange - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : +- ReusedSubquery : : : : : +- CometBroadcastExchange : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.item @@ -328,11 +292,7 @@ CometNativeColumnarToRow : : : : : : +- CometBroadcastHashJoin : : : : : : :- CometFilter : : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : : : : +- CometSubqueryBroadcast - : : : : : : : +- CometBroadcastExchange - : : : : : : : +- CometProject - : : : : : : : +- CometFilter - : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : : +- ReusedSubquery : : : : : : +- CometBroadcastExchange : : : : : : +- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.item @@ -351,11 +311,7 @@ CometNativeColumnarToRow : : : : +- CometBroadcastHashJoin : : : : :- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : : : +- CometSubqueryBroadcast - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- ReusedSubquery : : : : +- CometBroadcastExchange : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.item @@ -379,11 +335,7 @@ CometNativeColumnarToRow : : :- CometBroadcastHashJoin : : : :- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : : +- CometSubqueryBroadcast - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- ReusedSubquery : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometBroadcastHashJoin @@ -416,11 +368,7 @@ CometNativeColumnarToRow : : : : : : +- CometBroadcastHashJoin : : : : : : :- CometFilter : : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : : : : +- CometSubqueryBroadcast - : : : : : : : +- CometBroadcastExchange - : : : : : : : +- CometProject - : : : : : : : +- CometFilter - : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : : +- ReusedSubquery : : : : : : +- CometBroadcastExchange : : : : : : +- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.item @@ -439,11 +387,7 @@ CometNativeColumnarToRow : : : : +- CometBroadcastHashJoin : : : : :- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : : : +- CometSubqueryBroadcast - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- ReusedSubquery : : : : +- CometBroadcastExchange : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.item @@ -487,11 +431,7 @@ CometNativeColumnarToRow : : : : : +- CometBroadcastHashJoin : : : : : :- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : : : +- CometSubqueryBroadcast - : : : : : : +- CometBroadcastExchange - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : +- ReusedSubquery : : : : : +- CometBroadcastExchange : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.item @@ -510,11 +450,7 @@ CometNativeColumnarToRow : : : +- CometBroadcastHashJoin : : : :- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : : +- CometSubqueryBroadcast - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- ReusedSubquery : : : +- CometBroadcastExchange : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.item @@ -543,11 +479,7 @@ CometNativeColumnarToRow : : : :- CometProject : : : : +- CometBroadcastHashJoin : : : : :- CometNativeScan parquet spark_catalog.default.store_sales - : : : : : +- CometSubqueryBroadcast - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- ReusedSubquery : : : : +- CometBroadcastExchange : : : : +- CometProject : : : : +- CometFilter @@ -567,11 +499,7 @@ CometNativeColumnarToRow : : : +- CometProject : : : +- CometBroadcastHashJoin : : : :- CometNativeScan parquet spark_catalog.default.web_sales - : : : : +- CometSubqueryBroadcast - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- ReusedSubquery : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter @@ -623,11 +551,7 @@ CometNativeColumnarToRow : : : : : : : +- CometBroadcastHashJoin : : : : : : : :- CometFilter : : : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : : : : : +- CometSubqueryBroadcast - : : : : : : : : +- CometBroadcastExchange - : : : : : : : : +- CometProject - : : : : : : : : +- CometFilter - : : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : : : +- ReusedSubquery : : : : : : : +- CometBroadcastExchange : : : : : : : +- CometFilter : : : : : : : +- CometNativeScan parquet spark_catalog.default.item @@ -646,11 +570,7 @@ CometNativeColumnarToRow : : : : : +- CometBroadcastHashJoin : : : : : :- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : : : : +- CometSubqueryBroadcast - : : : : : : +- CometBroadcastExchange - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : +- ReusedSubquery : : : : : +- CometBroadcastExchange : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.item @@ -694,11 +614,7 @@ CometNativeColumnarToRow : : : : : : +- CometBroadcastHashJoin : : : : : : :- CometFilter : : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : : : : +- CometSubqueryBroadcast - : : : : : : : +- CometBroadcastExchange - : : : : : : : +- CometProject - : : : : : : : +- CometFilter - : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : : +- ReusedSubquery : : : : : : +- CometBroadcastExchange : : : : : : +- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.item @@ -717,11 +633,7 @@ CometNativeColumnarToRow : : : : +- CometBroadcastHashJoin : : : : :- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : : : +- CometSubqueryBroadcast - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- ReusedSubquery : : : : +- CometBroadcastExchange : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.item @@ -745,11 +657,7 @@ CometNativeColumnarToRow : : : :- CometBroadcastHashJoin : : : : :- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : : +- CometSubqueryBroadcast - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- ReusedSubquery : : : : +- CometBroadcastExchange : : : : +- CometProject : : : : +- CometBroadcastHashJoin @@ -782,11 +690,7 @@ CometNativeColumnarToRow : : : : : : : +- CometBroadcastHashJoin : : : : : : : :- CometFilter : : : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : : : : : +- CometSubqueryBroadcast - : : : : : : : : +- CometBroadcastExchange - : : : : : : : : +- CometProject - : : : : : : : : +- CometFilter - : : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : : : +- ReusedSubquery : : : : : : : +- CometBroadcastExchange : : : : : : : +- CometFilter : : : : : : : +- CometNativeScan parquet spark_catalog.default.item @@ -805,11 +709,7 @@ CometNativeColumnarToRow : : : : : +- CometBroadcastHashJoin : : : : : :- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : : : : +- CometSubqueryBroadcast - : : : : : : +- CometBroadcastExchange - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : +- ReusedSubquery : : : : : +- CometBroadcastExchange : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.item @@ -853,11 +753,7 @@ CometNativeColumnarToRow : : : : : : +- CometBroadcastHashJoin : : : : : : :- CometFilter : : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : : : : +- CometSubqueryBroadcast - : : : : : : : +- CometBroadcastExchange - : : : : : : : +- CometProject - : : : : : : : +- CometFilter - : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : : +- ReusedSubquery : : : : : : +- CometBroadcastExchange : : : : : : +- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.item @@ -876,11 +772,7 @@ CometNativeColumnarToRow : : : : +- CometBroadcastHashJoin : : : : :- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : : : +- CometSubqueryBroadcast - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- ReusedSubquery : : : : +- CometBroadcastExchange : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.item @@ -904,11 +796,7 @@ CometNativeColumnarToRow : : :- CometBroadcastHashJoin : : : :- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : : +- CometSubqueryBroadcast - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- ReusedSubquery : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometBroadcastHashJoin @@ -941,11 +829,7 @@ CometNativeColumnarToRow : : : : : : +- CometBroadcastHashJoin : : : : : : :- CometFilter : : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : : : : +- CometSubqueryBroadcast - : : : : : : : +- CometBroadcastExchange - : : : : : : : +- CometProject - : : : : : : : +- CometFilter - : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : : +- ReusedSubquery : : : : : : +- CometBroadcastExchange : : : : : : +- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.item @@ -964,11 +848,7 @@ CometNativeColumnarToRow : : : : +- CometBroadcastHashJoin : : : : :- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : : : +- CometSubqueryBroadcast - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- ReusedSubquery : : : : +- CometBroadcastExchange : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.item @@ -1012,11 +892,7 @@ CometNativeColumnarToRow : : : : : +- CometBroadcastHashJoin : : : : : :- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : : : +- CometSubqueryBroadcast - : : : : : : +- CometBroadcastExchange - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : +- ReusedSubquery : : : : : +- CometBroadcastExchange : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.item @@ -1035,11 +911,7 @@ CometNativeColumnarToRow : : : +- CometBroadcastHashJoin : : : :- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : : +- CometSubqueryBroadcast - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- ReusedSubquery : : : +- CometBroadcastExchange : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.item @@ -1068,11 +940,7 @@ CometNativeColumnarToRow : : : :- CometProject : : : : +- CometBroadcastHashJoin : : : : :- CometNativeScan parquet spark_catalog.default.store_sales - : : : : : +- CometSubqueryBroadcast - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- ReusedSubquery : : : : +- CometBroadcastExchange : : : : +- CometProject : : : : +- CometFilter @@ -1092,11 +960,7 @@ CometNativeColumnarToRow : : : +- CometProject : : : +- CometBroadcastHashJoin : : : :- CometNativeScan parquet spark_catalog.default.web_sales - : : : : +- CometSubqueryBroadcast - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- ReusedSubquery : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter @@ -1148,11 +1012,7 @@ CometNativeColumnarToRow : : : : : : : +- CometBroadcastHashJoin : : : : : : : :- CometFilter : : : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : : : : : +- CometSubqueryBroadcast - : : : : : : : : +- CometBroadcastExchange - : : : : : : : : +- CometProject - : : : : : : : : +- CometFilter - : : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : : : +- ReusedSubquery : : : : : : : +- CometBroadcastExchange : : : : : : : +- CometFilter : : : : : : : +- CometNativeScan parquet spark_catalog.default.item @@ -1171,11 +1031,7 @@ CometNativeColumnarToRow : : : : : +- CometBroadcastHashJoin : : : : : :- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : : : : +- CometSubqueryBroadcast - : : : : : : +- CometBroadcastExchange - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : +- ReusedSubquery : : : : : +- CometBroadcastExchange : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.item @@ -1219,11 +1075,7 @@ CometNativeColumnarToRow : : : : : : +- CometBroadcastHashJoin : : : : : : :- CometFilter : : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : : : : +- CometSubqueryBroadcast - : : : : : : : +- CometBroadcastExchange - : : : : : : : +- CometProject - : : : : : : : +- CometFilter - : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : : +- ReusedSubquery : : : : : : +- CometBroadcastExchange : : : : : : +- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.item @@ -1242,11 +1094,7 @@ CometNativeColumnarToRow : : : : +- CometBroadcastHashJoin : : : : :- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : : : +- CometSubqueryBroadcast - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- ReusedSubquery : : : : +- CometBroadcastExchange : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.item @@ -1270,11 +1118,7 @@ CometNativeColumnarToRow : : : :- CometBroadcastHashJoin : : : : :- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : : +- CometSubqueryBroadcast - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- ReusedSubquery : : : : +- CometBroadcastExchange : : : : +- CometProject : : : : +- CometBroadcastHashJoin @@ -1307,11 +1151,7 @@ CometNativeColumnarToRow : : : : : : : +- CometBroadcastHashJoin : : : : : : : :- CometFilter : : : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : : : : : +- CometSubqueryBroadcast - : : : : : : : : +- CometBroadcastExchange - : : : : : : : : +- CometProject - : : : : : : : : +- CometFilter - : : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : : : +- ReusedSubquery : : : : : : : +- CometBroadcastExchange : : : : : : : +- CometFilter : : : : : : : +- CometNativeScan parquet spark_catalog.default.item @@ -1330,11 +1170,7 @@ CometNativeColumnarToRow : : : : : +- CometBroadcastHashJoin : : : : : :- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : : : : +- CometSubqueryBroadcast - : : : : : : +- CometBroadcastExchange - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : +- ReusedSubquery : : : : : +- CometBroadcastExchange : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.item @@ -1378,11 +1214,7 @@ CometNativeColumnarToRow : : : : : : +- CometBroadcastHashJoin : : : : : : :- CometFilter : : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : : : : +- CometSubqueryBroadcast - : : : : : : : +- CometBroadcastExchange - : : : : : : : +- CometProject - : : : : : : : +- CometFilter - : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : : +- ReusedSubquery : : : : : : +- CometBroadcastExchange : : : : : : +- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.item @@ -1401,11 +1233,7 @@ CometNativeColumnarToRow : : : : +- CometBroadcastHashJoin : : : : :- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : : : +- CometSubqueryBroadcast - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- ReusedSubquery : : : : +- CometBroadcastExchange : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.item @@ -1429,11 +1257,7 @@ CometNativeColumnarToRow : : :- CometBroadcastHashJoin : : : :- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : : +- CometSubqueryBroadcast - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- ReusedSubquery : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometBroadcastHashJoin @@ -1466,11 +1290,7 @@ CometNativeColumnarToRow : : : : : : +- CometBroadcastHashJoin : : : : : : :- CometFilter : : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : : : : +- CometSubqueryBroadcast - : : : : : : : +- CometBroadcastExchange - : : : : : : : +- CometProject - : : : : : : : +- CometFilter - : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : : +- ReusedSubquery : : : : : : +- CometBroadcastExchange : : : : : : +- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.item @@ -1489,11 +1309,7 @@ CometNativeColumnarToRow : : : : +- CometBroadcastHashJoin : : : : :- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : : : +- CometSubqueryBroadcast - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- ReusedSubquery : : : : +- CometBroadcastExchange : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.item @@ -1537,11 +1353,7 @@ CometNativeColumnarToRow : : : : : +- CometBroadcastHashJoin : : : : : :- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : : : +- CometSubqueryBroadcast - : : : : : : +- CometBroadcastExchange - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : +- ReusedSubquery : : : : : +- CometBroadcastExchange : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.item @@ -1560,11 +1372,7 @@ CometNativeColumnarToRow : : : +- CometBroadcastHashJoin : : : :- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : : +- CometSubqueryBroadcast - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- ReusedSubquery : : : +- CometBroadcastExchange : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.item @@ -1593,11 +1401,7 @@ CometNativeColumnarToRow : : : :- CometProject : : : : +- CometBroadcastHashJoin : : : : :- CometNativeScan parquet spark_catalog.default.store_sales - : : : : : +- CometSubqueryBroadcast - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- ReusedSubquery : : : : +- CometBroadcastExchange : : : : +- CometProject : : : : +- CometFilter @@ -1617,11 +1421,7 @@ CometNativeColumnarToRow : : : +- CometProject : : : +- CometBroadcastHashJoin : : : :- CometNativeScan parquet spark_catalog.default.web_sales - : : : : +- CometSubqueryBroadcast - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- ReusedSubquery : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter @@ -1673,11 +1473,7 @@ CometNativeColumnarToRow : : : : : : : +- CometBroadcastHashJoin : : : : : : : :- CometFilter : : : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : : : : : +- CometSubqueryBroadcast - : : : : : : : : +- CometBroadcastExchange - : : : : : : : : +- CometProject - : : : : : : : : +- CometFilter - : : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : : : +- ReusedSubquery : : : : : : : +- CometBroadcastExchange : : : : : : : +- CometFilter : : : : : : : +- CometNativeScan parquet spark_catalog.default.item @@ -1696,11 +1492,7 @@ CometNativeColumnarToRow : : : : : +- CometBroadcastHashJoin : : : : : :- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : : : : +- CometSubqueryBroadcast - : : : : : : +- CometBroadcastExchange - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : +- ReusedSubquery : : : : : +- CometBroadcastExchange : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.item @@ -1744,11 +1536,7 @@ CometNativeColumnarToRow : : : : : : +- CometBroadcastHashJoin : : : : : : :- CometFilter : : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : : : : +- CometSubqueryBroadcast - : : : : : : : +- CometBroadcastExchange - : : : : : : : +- CometProject - : : : : : : : +- CometFilter - : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : : +- ReusedSubquery : : : : : : +- CometBroadcastExchange : : : : : : +- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.item @@ -1767,11 +1555,7 @@ CometNativeColumnarToRow : : : : +- CometBroadcastHashJoin : : : : :- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : : : +- CometSubqueryBroadcast - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- ReusedSubquery : : : : +- CometBroadcastExchange : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.item @@ -1795,11 +1579,7 @@ CometNativeColumnarToRow : : : :- CometBroadcastHashJoin : : : : :- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : : +- CometSubqueryBroadcast - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- ReusedSubquery : : : : +- CometBroadcastExchange : : : : +- CometProject : : : : +- CometBroadcastHashJoin @@ -1832,11 +1612,7 @@ CometNativeColumnarToRow : : : : : : : +- CometBroadcastHashJoin : : : : : : : :- CometFilter : : : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : : : : : +- CometSubqueryBroadcast - : : : : : : : : +- CometBroadcastExchange - : : : : : : : : +- CometProject - : : : : : : : : +- CometFilter - : : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : : : +- ReusedSubquery : : : : : : : +- CometBroadcastExchange : : : : : : : +- CometFilter : : : : : : : +- CometNativeScan parquet spark_catalog.default.item @@ -1855,11 +1631,7 @@ CometNativeColumnarToRow : : : : : +- CometBroadcastHashJoin : : : : : :- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : : : : +- CometSubqueryBroadcast - : : : : : : +- CometBroadcastExchange - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : +- ReusedSubquery : : : : : +- CometBroadcastExchange : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.item @@ -1903,11 +1675,7 @@ CometNativeColumnarToRow : : : : : : +- CometBroadcastHashJoin : : : : : : :- CometFilter : : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : : : : +- CometSubqueryBroadcast - : : : : : : : +- CometBroadcastExchange - : : : : : : : +- CometProject - : : : : : : : +- CometFilter - : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : : +- ReusedSubquery : : : : : : +- CometBroadcastExchange : : : : : : +- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.item @@ -1926,11 +1694,7 @@ CometNativeColumnarToRow : : : : +- CometBroadcastHashJoin : : : : :- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : : : +- CometSubqueryBroadcast - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- ReusedSubquery : : : : +- CometBroadcastExchange : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.item @@ -1954,11 +1718,7 @@ CometNativeColumnarToRow : : :- CometBroadcastHashJoin : : : :- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : : +- CometSubqueryBroadcast - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- ReusedSubquery : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometBroadcastHashJoin @@ -1991,11 +1751,7 @@ CometNativeColumnarToRow : : : : : : +- CometBroadcastHashJoin : : : : : : :- CometFilter : : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : : : : +- CometSubqueryBroadcast - : : : : : : : +- CometBroadcastExchange - : : : : : : : +- CometProject - : : : : : : : +- CometFilter - : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : : +- ReusedSubquery : : : : : : +- CometBroadcastExchange : : : : : : +- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.item @@ -2014,11 +1770,7 @@ CometNativeColumnarToRow : : : : +- CometBroadcastHashJoin : : : : :- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : : : +- CometSubqueryBroadcast - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- ReusedSubquery : : : : +- CometBroadcastExchange : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.item @@ -2062,11 +1814,7 @@ CometNativeColumnarToRow : : : : : +- CometBroadcastHashJoin : : : : : :- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : : : +- CometSubqueryBroadcast - : : : : : : +- CometBroadcastExchange - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : +- ReusedSubquery : : : : : +- CometBroadcastExchange : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.item @@ -2085,11 +1833,7 @@ CometNativeColumnarToRow : : : +- CometBroadcastHashJoin : : : :- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : : +- CometSubqueryBroadcast - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- ReusedSubquery : : : +- CometBroadcastExchange : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.item @@ -2118,11 +1862,7 @@ CometNativeColumnarToRow : : :- CometProject : : : +- CometBroadcastHashJoin : : : :- CometNativeScan parquet spark_catalog.default.store_sales - : : : : +- CometSubqueryBroadcast - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- ReusedSubquery : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter @@ -2142,11 +1882,7 @@ CometNativeColumnarToRow : : +- CometProject : : +- CometBroadcastHashJoin : : :- CometNativeScan parquet spark_catalog.default.web_sales - : : : +- CometSubqueryBroadcast - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- ReusedSubquery : : +- CometBroadcastExchange : : +- CometProject : : +- CometFilter @@ -2198,11 +1934,7 @@ CometNativeColumnarToRow : : : : : : +- CometBroadcastHashJoin : : : : : : :- CometFilter : : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : : : : +- CometSubqueryBroadcast - : : : : : : : +- CometBroadcastExchange - : : : : : : : +- CometProject - : : : : : : : +- CometFilter - : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : : +- ReusedSubquery : : : : : : +- CometBroadcastExchange : : : : : : +- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.item @@ -2221,11 +1953,7 @@ CometNativeColumnarToRow : : : : +- CometBroadcastHashJoin : : : : :- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : : : +- CometSubqueryBroadcast - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- ReusedSubquery : : : : +- CometBroadcastExchange : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.item @@ -2269,11 +1997,7 @@ CometNativeColumnarToRow : : : : : +- CometBroadcastHashJoin : : : : : :- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : : : +- CometSubqueryBroadcast - : : : : : : +- CometBroadcastExchange - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : +- ReusedSubquery : : : : : +- CometBroadcastExchange : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.item @@ -2292,11 +2016,7 @@ CometNativeColumnarToRow : : : +- CometBroadcastHashJoin : : : :- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : : +- CometSubqueryBroadcast - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- ReusedSubquery : : : +- CometBroadcastExchange : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.item @@ -2320,11 +2040,7 @@ CometNativeColumnarToRow : : :- CometBroadcastHashJoin : : : :- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : +- CometSubqueryBroadcast - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- ReusedSubquery : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometBroadcastHashJoin @@ -2357,11 +2073,7 @@ CometNativeColumnarToRow : : : : : : +- CometBroadcastHashJoin : : : : : : :- CometFilter : : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : : : : +- CometSubqueryBroadcast - : : : : : : : +- CometBroadcastExchange - : : : : : : : +- CometProject - : : : : : : : +- CometFilter - : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : : +- ReusedSubquery : : : : : : +- CometBroadcastExchange : : : : : : +- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.item @@ -2380,11 +2092,7 @@ CometNativeColumnarToRow : : : : +- CometBroadcastHashJoin : : : : :- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : : : +- CometSubqueryBroadcast - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- ReusedSubquery : : : : +- CometBroadcastExchange : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.item @@ -2428,11 +2136,7 @@ CometNativeColumnarToRow : : : : : +- CometBroadcastHashJoin : : : : : :- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : : : +- CometSubqueryBroadcast - : : : : : : +- CometBroadcastExchange - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : +- ReusedSubquery : : : : : +- CometBroadcastExchange : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.item @@ -2451,11 +2155,7 @@ CometNativeColumnarToRow : : : +- CometBroadcastHashJoin : : : :- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : : +- CometSubqueryBroadcast - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- ReusedSubquery : : : +- CometBroadcastExchange : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.item @@ -2479,11 +2179,7 @@ CometNativeColumnarToRow : :- CometBroadcastHashJoin : : :- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : +- CometSubqueryBroadcast - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- ReusedSubquery : : +- CometBroadcastExchange : : +- CometProject : : +- CometBroadcastHashJoin @@ -2516,11 +2212,7 @@ CometNativeColumnarToRow : : : : : +- CometBroadcastHashJoin : : : : : :- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : : : +- CometSubqueryBroadcast - : : : : : : +- CometBroadcastExchange - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : +- ReusedSubquery : : : : : +- CometBroadcastExchange : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.item @@ -2539,11 +2231,7 @@ CometNativeColumnarToRow : : : +- CometBroadcastHashJoin : : : :- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : : +- CometSubqueryBroadcast - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- ReusedSubquery : : : +- CometBroadcastExchange : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.item @@ -2587,11 +2275,7 @@ CometNativeColumnarToRow : : : : +- CometBroadcastHashJoin : : : : :- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : : +- CometSubqueryBroadcast - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- ReusedSubquery : : : : +- CometBroadcastExchange : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.item @@ -2610,11 +2294,7 @@ CometNativeColumnarToRow : : +- CometBroadcastHashJoin : : :- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : +- CometSubqueryBroadcast - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- ReusedSubquery : : +- CometBroadcastExchange : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.item @@ -2627,4 +2307,4 @@ CometNativeColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.date_dim -Comet accelerated 2487 out of 2622 eligible operators (94%). Final plan contains 6 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 2167 out of 2302 eligible operators (94%). Final plan contains 6 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q18a.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q18a.native_datafusion/extended.txt index b34ebfdf84..75f132f6fb 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q18a.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q18a.native_datafusion/extended.txt @@ -63,11 +63,7 @@ CometNativeColumnarToRow : : : : : : +- CometBroadcastHashJoin : : : : : : :- CometFilter : : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : : : : +- CometSubqueryBroadcast - : : : : : : : +- CometBroadcastExchange - : : : : : : : +- CometProject - : : : : : : : +- CometFilter - : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : : +- ReusedSubquery : : : : : : +- CometBroadcastExchange : : : : : : +- CometProject : : : : : : +- CometFilter @@ -108,11 +104,7 @@ CometNativeColumnarToRow : : : : : : +- CometBroadcastHashJoin : : : : : : :- CometFilter : : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : : : : +- CometSubqueryBroadcast - : : : : : : : +- CometBroadcastExchange - : : : : : : : +- CometProject - : : : : : : : +- CometFilter - : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : : +- ReusedSubquery : : : : : : +- CometBroadcastExchange : : : : : : +- CometProject : : : : : : +- CometFilter @@ -153,11 +145,7 @@ CometNativeColumnarToRow : : : : : : +- CometBroadcastHashJoin : : : : : : :- CometFilter : : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : : : : +- CometSubqueryBroadcast - : : : : : : : +- CometBroadcastExchange - : : : : : : : +- CometProject - : : : : : : : +- CometFilter - : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : : +- ReusedSubquery : : : : : : +- CometBroadcastExchange : : : : : : +- CometProject : : : : : : +- CometFilter @@ -198,11 +186,7 @@ CometNativeColumnarToRow : : : : : +- CometBroadcastHashJoin : : : : : :- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : : : +- CometSubqueryBroadcast - : : : : : : +- CometBroadcastExchange - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : +- ReusedSubquery : : : : : +- CometBroadcastExchange : : : : : +- CometProject : : : : : +- CometFilter @@ -226,4 +210,4 @@ CometNativeColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.item -Comet accelerated 221 out of 226 eligible operators (97%). Final plan contains 1 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 205 out of 210 eligible operators (97%). Final plan contains 1 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q27a.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q27a.native_datafusion/extended.txt index 9c4497cbd3..70614792ba 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q27a.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q27a.native_datafusion/extended.txt @@ -48,11 +48,7 @@ CometNativeColumnarToRow : : : : +- CometBroadcastHashJoin : : : : :- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : : : +- CometSubqueryBroadcast - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- ReusedSubquery : : : : +- CometBroadcastExchange : : : : +- CometProject : : : : +- CometFilter @@ -82,11 +78,7 @@ CometNativeColumnarToRow : : : +- CometBroadcastHashJoin : : : :- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : : +- CometSubqueryBroadcast - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- ReusedSubquery : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter @@ -103,4 +95,4 @@ CometNativeColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.item -Comet accelerated 100 out of 103 eligible operators (97%). Final plan contains 1 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 92 out of 95 eligible operators (96%). Final plan contains 1 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q35.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q35.native_datafusion/extended.txt index 2e79f6f464..b7c8209556 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q35.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q35.native_datafusion/extended.txt @@ -33,11 +33,7 @@ TakeOrderedAndProject : : : +- CometProject : : : +- CometBroadcastHashJoin : : : :- CometNativeScan parquet spark_catalog.default.web_sales - : : : : +- CometSubqueryBroadcast - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- ReusedSubquery : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter @@ -47,11 +43,7 @@ TakeOrderedAndProject : : +- CometProject : : +- CometBroadcastHashJoin : : :- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : +- CometSubqueryBroadcast - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- ReusedSubquery : : +- CometBroadcastExchange : : +- CometProject : : +- CometFilter @@ -67,4 +59,4 @@ TakeOrderedAndProject +- CometFilter +- CometNativeScan parquet spark_catalog.default.customer_demographics -Comet accelerated 44 out of 62 eligible operators (70%). Final plan contains 6 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 36 out of 54 eligible operators (66%). Final plan contains 6 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q35a.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q35a.native_datafusion/extended.txt index 173c450c71..f27de2fdbb 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q35a.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q35a.native_datafusion/extended.txt @@ -30,11 +30,7 @@ CometNativeColumnarToRow : : :- CometProject : : : +- CometBroadcastHashJoin : : : :- CometNativeScan parquet spark_catalog.default.web_sales - : : : : +- CometSubqueryBroadcast - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- ReusedSubquery : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter @@ -42,11 +38,7 @@ CometNativeColumnarToRow : : +- CometProject : : +- CometBroadcastHashJoin : : :- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : +- CometSubqueryBroadcast - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- ReusedSubquery : : +- CometBroadcastExchange : : +- CometProject : : +- CometFilter @@ -60,4 +52,4 @@ CometNativeColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.customer_demographics -Comet accelerated 57 out of 60 eligible operators (95%). Final plan contains 1 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 49 out of 52 eligible operators (94%). Final plan contains 1 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q49.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q49.native_datafusion/extended.txt index f2ae44fb86..cd6c39e4f3 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q49.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q49.native_datafusion/extended.txt @@ -54,11 +54,7 @@ CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : +- CometSubqueryBroadcast - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- ReusedSubquery : : +- CometProject : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.catalog_returns @@ -85,11 +81,7 @@ CometNativeColumnarToRow : : +- CometProject : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : +- CometSubqueryBroadcast - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- ReusedSubquery : +- CometProject : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.store_returns @@ -98,4 +90,4 @@ CometNativeColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.date_dim -Comet accelerated 75 out of 95 eligible operators (78%). Final plan contains 4 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 67 out of 87 eligible operators (77%). Final plan contains 4 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q51a.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q51a.native_datafusion/extended.txt index a3400c5ecf..c8010c1286 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q51a.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q51a.native_datafusion/extended.txt @@ -82,11 +82,7 @@ TakeOrderedAndProject : : +- CometBroadcastHashJoin : : :- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : +- CometSubqueryBroadcast - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- ReusedSubquery : : +- CometBroadcastExchange : : +- CometProject : : +- CometFilter @@ -104,11 +100,7 @@ TakeOrderedAndProject : +- CometBroadcastHashJoin : :- CometFilter : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : +- CometSubqueryBroadcast - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- ReusedSubquery : +- CometBroadcastExchange : +- CometProject : +- CometFilter @@ -193,11 +185,7 @@ TakeOrderedAndProject : +- CometBroadcastHashJoin : :- CometFilter : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : +- CometSubqueryBroadcast - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- ReusedSubquery : +- CometBroadcastExchange : +- CometProject : +- CometFilter @@ -215,14 +203,10 @@ TakeOrderedAndProject +- CometBroadcastHashJoin :- CometFilter : +- CometNativeScan parquet spark_catalog.default.store_sales - : +- CometSubqueryBroadcast - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- ReusedSubquery +- CometBroadcastExchange +- CometProject +- CometFilter +- CometNativeScan parquet spark_catalog.default.date_dim -Comet accelerated 158 out of 212 eligible operators (74%). Final plan contains 14 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 142 out of 196 eligible operators (72%). Final plan contains 14 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q5a.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q5a.native_datafusion/extended.txt index b472530c10..8d61d1e1ad 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q5a.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q5a.native_datafusion/extended.txt @@ -27,11 +27,7 @@ CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.store_returns - : : : : +- CometSubqueryBroadcast - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- ReusedSubquery : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter @@ -51,19 +47,11 @@ CometNativeColumnarToRow : : : : :- CometProject : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : : +- CometSubqueryBroadcast - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- ReusedSubquery : : : : +- CometProject : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.catalog_returns - : : : : +- CometSubqueryBroadcast - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- ReusedSubquery : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter @@ -83,20 +71,12 @@ CometNativeColumnarToRow : : : :- CometProject : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : : +- CometSubqueryBroadcast - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- ReusedSubquery : : : +- CometProject : : : +- CometBroadcastHashJoin : : : :- CometBroadcastExchange : : : : +- CometNativeScan parquet spark_catalog.default.web_returns - : : : : +- CometSubqueryBroadcast - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- ReusedSubquery : : : +- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.web_sales @@ -134,11 +114,7 @@ CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.store_returns - : : : : +- CometSubqueryBroadcast - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- ReusedSubquery : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter @@ -158,19 +134,11 @@ CometNativeColumnarToRow : : : : :- CometProject : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : : +- CometSubqueryBroadcast - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- ReusedSubquery : : : : +- CometProject : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.catalog_returns - : : : : +- CometSubqueryBroadcast - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- ReusedSubquery : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter @@ -190,20 +158,12 @@ CometNativeColumnarToRow : : : :- CometProject : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : : +- CometSubqueryBroadcast - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- ReusedSubquery : : : +- CometProject : : : +- CometBroadcastHashJoin : : : :- CometBroadcastExchange : : : : +- CometNativeScan parquet spark_catalog.default.web_returns - : : : : +- CometSubqueryBroadcast - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- ReusedSubquery : : : +- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.web_sales @@ -241,11 +201,7 @@ CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.store_returns - : : : +- CometSubqueryBroadcast - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- ReusedSubquery : : +- CometBroadcastExchange : : +- CometProject : : +- CometFilter @@ -265,19 +221,11 @@ CometNativeColumnarToRow : : : :- CometProject : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : +- CometSubqueryBroadcast - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- ReusedSubquery : : : +- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.catalog_returns - : : : +- CometSubqueryBroadcast - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- ReusedSubquery : : +- CometBroadcastExchange : : +- CometProject : : +- CometFilter @@ -297,20 +245,12 @@ CometNativeColumnarToRow : : :- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : +- CometSubqueryBroadcast - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- ReusedSubquery : : +- CometProject : : +- CometBroadcastHashJoin : : :- CometBroadcastExchange : : : +- CometNativeScan parquet spark_catalog.default.web_returns - : : : +- CometSubqueryBroadcast - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- ReusedSubquery : : +- CometProject : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.web_sales @@ -323,4 +263,4 @@ CometNativeColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.web_site -Comet accelerated 305 out of 323 eligible operators (94%). Final plan contains 1 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 245 out of 263 eligible operators (93%). Final plan contains 1 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q6.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q6.native_datafusion/extended.txt index e8759a6e10..dd68f2476e 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q6.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q6.native_datafusion/extended.txt @@ -25,14 +25,7 @@ CometNativeColumnarToRow : : +- CometBroadcastExchange : : +- CometProject : : +- CometFilter - : : : +- Subquery - : : : +- CometNativeColumnarToRow - : : : +- CometHashAggregate - : : : +- CometExchange - : : : +- CometHashAggregate - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- ReusedSubquery : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- Subquery : : +- CometNativeColumnarToRow @@ -69,4 +62,4 @@ CometNativeColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.item -Comet accelerated 61 out of 66 eligible operators (92%). Final plan contains 4 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 55 out of 60 eligible operators (91%). Final plan contains 3 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q70a.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q70a.native_datafusion/extended.txt index 97dd294f61..a60ad5ebc5 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q70a.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q70a.native_datafusion/extended.txt @@ -50,11 +50,7 @@ TakeOrderedAndProject : : +- CometBroadcastHashJoin : : :- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : +- CometSubqueryBroadcast - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- ReusedSubquery : : +- CometBroadcastExchange : : +- CometProject : : +- CometFilter @@ -109,11 +105,7 @@ TakeOrderedAndProject : : +- CometBroadcastHashJoin : : :- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : +- CometSubqueryBroadcast - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- ReusedSubquery : : +- CometBroadcastExchange : : +- CometProject : : +- CometFilter @@ -168,11 +160,7 @@ TakeOrderedAndProject : +- CometBroadcastHashJoin : :- CometFilter : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : +- CometSubqueryBroadcast - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- ReusedSubquery : +- CometBroadcastExchange : +- CometProject : +- CometFilter @@ -182,4 +170,4 @@ TakeOrderedAndProject +- CometFilter +- CometNativeScan parquet spark_catalog.default.date_dim -Comet accelerated 117 out of 168 eligible operators (69%). Final plan contains 15 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 105 out of 156 eligible operators (67%). Final plan contains 15 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q74.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q74.native_datafusion/extended.txt index 7768138645..cfe634568f 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q74.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q74.native_datafusion/extended.txt @@ -62,10 +62,7 @@ CometNativeColumnarToRow : : +- CometBroadcastExchange : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : +- CometSubqueryBroadcast - : : +- CometBroadcastExchange - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- ReusedSubquery : +- CometBroadcastExchange : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -83,12 +80,9 @@ CometNativeColumnarToRow : +- CometBroadcastExchange : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.web_sales - : +- CometSubqueryBroadcast - : +- CometBroadcastExchange - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- ReusedSubquery +- CometBroadcastExchange +- CometFilter +- CometNativeScan parquet spark_catalog.default.date_dim -Comet accelerated 87 out of 91 eligible operators (95%). Final plan contains 1 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 81 out of 85 eligible operators (95%). Final plan contains 1 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q75.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q75.native_datafusion/extended.txt index ee7c0af216..bccd23b141 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q75.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q75.native_datafusion/extended.txt @@ -48,10 +48,7 @@ CometNativeColumnarToRow : : : : +- CometBroadcastHashJoin : : : : :- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : : : +- CometSubqueryBroadcast - : : : : : +- CometBroadcastExchange - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- ReusedSubquery : : : : +- CometBroadcastExchange : : : : +- CometProject : : : : +- CometFilter @@ -74,10 +71,7 @@ CometNativeColumnarToRow : : : +- CometBroadcastHashJoin : : : :- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : : +- CometSubqueryBroadcast - : : : : +- CometBroadcastExchange - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- ReusedSubquery : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter @@ -136,10 +130,7 @@ CometNativeColumnarToRow : : : +- CometBroadcastHashJoin : : : :- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : : +- CometSubqueryBroadcast - : : : : +- CometBroadcastExchange - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- ReusedSubquery : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter @@ -162,10 +153,7 @@ CometNativeColumnarToRow : : +- CometBroadcastHashJoin : : :- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : +- CometSubqueryBroadcast - : : : +- CometBroadcastExchange - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- ReusedSubquery : : +- CometBroadcastExchange : : +- CometProject : : +- CometFilter @@ -179,4 +167,4 @@ CometNativeColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.web_returns -Comet accelerated 173 out of 179 eligible operators (96%). Final plan contains 1 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 161 out of 167 eligible operators (96%). Final plan contains 1 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q77a.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q77a.native_datafusion/extended.txt index 6621cfd8f8..95b315d258 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q77a.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q77a.native_datafusion/extended.txt @@ -43,11 +43,7 @@ CometNativeColumnarToRow : : : +- CometBroadcastHashJoin : : : :- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.store_returns - : : : : +- CometSubqueryBroadcast - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- ReusedSubquery : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter @@ -65,11 +61,7 @@ CometNativeColumnarToRow : : : +- CometProject : : : +- CometBroadcastHashJoin : : : :- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : +- CometSubqueryBroadcast - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- ReusedSubquery : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter @@ -81,11 +73,7 @@ CometNativeColumnarToRow : : +- CometProject : : +- CometBroadcastHashJoin : : :- CometNativeScan parquet spark_catalog.default.catalog_returns - : : : +- CometSubqueryBroadcast - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- ReusedSubquery : : +- CometBroadcastExchange : : +- CometProject : : +- CometFilter @@ -102,11 +90,7 @@ CometNativeColumnarToRow : : : +- CometBroadcastHashJoin : : : :- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : : +- CometSubqueryBroadcast - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- ReusedSubquery : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter @@ -124,11 +108,7 @@ CometNativeColumnarToRow : : +- CometBroadcastHashJoin : : :- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.web_returns - : : : +- CometSubqueryBroadcast - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- ReusedSubquery : : +- CometBroadcastExchange : : +- CometProject : : +- CometFilter @@ -179,11 +159,7 @@ CometNativeColumnarToRow : : : +- CometBroadcastHashJoin : : : :- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.store_returns - : : : : +- CometSubqueryBroadcast - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- ReusedSubquery : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter @@ -201,11 +177,7 @@ CometNativeColumnarToRow : : : +- CometProject : : : +- CometBroadcastHashJoin : : : :- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : +- CometSubqueryBroadcast - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- ReusedSubquery : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter @@ -217,11 +189,7 @@ CometNativeColumnarToRow : : +- CometProject : : +- CometBroadcastHashJoin : : :- CometNativeScan parquet spark_catalog.default.catalog_returns - : : : +- CometSubqueryBroadcast - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- ReusedSubquery : : +- CometBroadcastExchange : : +- CometProject : : +- CometFilter @@ -238,11 +206,7 @@ CometNativeColumnarToRow : : : +- CometBroadcastHashJoin : : : :- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : : +- CometSubqueryBroadcast - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- ReusedSubquery : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter @@ -260,11 +224,7 @@ CometNativeColumnarToRow : : +- CometBroadcastHashJoin : : :- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.web_returns - : : : +- CometSubqueryBroadcast - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- ReusedSubquery : : +- CometBroadcastExchange : : +- CometProject : : +- CometFilter @@ -315,11 +275,7 @@ CometNativeColumnarToRow : : +- CometBroadcastHashJoin : : :- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.store_returns - : : : +- CometSubqueryBroadcast - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- ReusedSubquery : : +- CometBroadcastExchange : : +- CometProject : : +- CometFilter @@ -337,11 +293,7 @@ CometNativeColumnarToRow : : +- CometProject : : +- CometBroadcastHashJoin : : :- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : +- CometSubqueryBroadcast - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- ReusedSubquery : : +- CometBroadcastExchange : : +- CometProject : : +- CometFilter @@ -353,11 +305,7 @@ CometNativeColumnarToRow : +- CometProject : +- CometBroadcastHashJoin : :- CometNativeScan parquet spark_catalog.default.catalog_returns - : : +- CometSubqueryBroadcast - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- ReusedSubquery : +- CometBroadcastExchange : +- CometProject : +- CometFilter @@ -374,11 +322,7 @@ CometNativeColumnarToRow : : +- CometBroadcastHashJoin : : :- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : +- CometSubqueryBroadcast - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- ReusedSubquery : : +- CometBroadcastExchange : : +- CometProject : : +- CometFilter @@ -396,11 +340,7 @@ CometNativeColumnarToRow : +- CometBroadcastHashJoin : :- CometFilter : : +- CometNativeScan parquet spark_catalog.default.web_returns - : : +- CometSubqueryBroadcast - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- ReusedSubquery : +- CometBroadcastExchange : +- CometProject : +- CometFilter @@ -409,4 +349,4 @@ CometNativeColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.web_page -Comet accelerated 350 out of 392 eligible operators (89%). Final plan contains 18 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 290 out of 332 eligible operators (87%). Final plan contains 18 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q78.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q78.native_datafusion/extended.txt index 8426b9653c..d29dbc13e5 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q78.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q78.native_datafusion/extended.txt @@ -43,10 +43,7 @@ TakeOrderedAndProject : : : +- CometExchange : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : +- CometSubqueryBroadcast - : : : +- CometBroadcastExchange - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- ReusedSubquery : : +- CometSort : : +- CometExchange : : +- CometProject @@ -69,10 +66,7 @@ TakeOrderedAndProject : : +- CometExchange : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : +- CometSubqueryBroadcast - : : +- CometBroadcastExchange - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- ReusedSubquery : +- CometSort : +- CometExchange : +- CometProject @@ -82,4 +76,4 @@ TakeOrderedAndProject +- CometFilter +- CometNativeScan parquet spark_catalog.default.date_dim -Comet accelerated 77 out of 82 eligible operators (93%). Final plan contains 1 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 71 out of 76 eligible operators (93%). Final plan contains 1 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q80a.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q80a.native_datafusion/extended.txt index dced05333a..10e0803953 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q80a.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q80a.native_datafusion/extended.txt @@ -68,11 +68,7 @@ CometNativeColumnarToRow : : : : : : : +- CometExchange : : : : : : : +- CometFilter : : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : : : : +- CometSubqueryBroadcast - : : : : : : : +- CometBroadcastExchange - : : : : : : : +- CometProject - : : : : : : : +- CometFilter - : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : : +- ReusedSubquery : : : : : : +- CometSort : : : : : : +- CometExchange : : : : : : +- CometProject @@ -111,11 +107,7 @@ CometNativeColumnarToRow : : : : : : +- CometExchange : : : : : : +- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : : : : +- CometSubqueryBroadcast - : : : : : : +- CometBroadcastExchange - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : +- ReusedSubquery : : : : : +- CometSort : : : : : +- CometExchange : : : : : +- CometProject @@ -204,11 +196,7 @@ CometNativeColumnarToRow : : : : : : : +- CometExchange : : : : : : : +- CometFilter : : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : : : : +- CometSubqueryBroadcast - : : : : : : : +- CometBroadcastExchange - : : : : : : : +- CometProject - : : : : : : : +- CometFilter - : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : : +- ReusedSubquery : : : : : : +- CometSort : : : : : : +- CometExchange : : : : : : +- CometProject @@ -247,11 +235,7 @@ CometNativeColumnarToRow : : : : : : +- CometExchange : : : : : : +- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : : : : +- CometSubqueryBroadcast - : : : : : : +- CometBroadcastExchange - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : +- ReusedSubquery : : : : : +- CometSort : : : : : +- CometExchange : : : : : +- CometProject @@ -340,11 +324,7 @@ CometNativeColumnarToRow : : : : : : +- CometExchange : : : : : : +- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : : : +- CometSubqueryBroadcast - : : : : : : +- CometBroadcastExchange - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : +- ReusedSubquery : : : : : +- CometSort : : : : : +- CometExchange : : : : : +- CometProject @@ -383,11 +363,7 @@ CometNativeColumnarToRow : : : : : +- CometExchange : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : : : +- CometSubqueryBroadcast - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- ReusedSubquery : : : : +- CometSort : : : : +- CometExchange : : : : +- CometProject @@ -410,4 +386,4 @@ CometNativeColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.promotion -Comet accelerated 401 out of 410 eligible operators (97%). Final plan contains 1 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 377 out of 386 eligible operators (97%). Final plan contains 1 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q10a.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q10a.native_datafusion/extended.txt index 173c450c71..f27de2fdbb 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q10a.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q10a.native_datafusion/extended.txt @@ -30,11 +30,7 @@ CometNativeColumnarToRow : : :- CometProject : : : +- CometBroadcastHashJoin : : : :- CometNativeScan parquet spark_catalog.default.web_sales - : : : : +- CometSubqueryBroadcast - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- ReusedSubquery : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter @@ -42,11 +38,7 @@ CometNativeColumnarToRow : : +- CometProject : : +- CometBroadcastHashJoin : : :- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : +- CometSubqueryBroadcast - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- ReusedSubquery : : +- CometBroadcastExchange : : +- CometProject : : +- CometFilter @@ -60,4 +52,4 @@ CometNativeColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.customer_demographics -Comet accelerated 57 out of 60 eligible operators (95%). Final plan contains 1 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 49 out of 52 eligible operators (94%). Final plan contains 1 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q11.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q11.native_datafusion/extended.txt index 7768138645..cfe634568f 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q11.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q11.native_datafusion/extended.txt @@ -62,10 +62,7 @@ CometNativeColumnarToRow : : +- CometBroadcastExchange : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : +- CometSubqueryBroadcast - : : +- CometBroadcastExchange - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- ReusedSubquery : +- CometBroadcastExchange : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -83,12 +80,9 @@ CometNativeColumnarToRow : +- CometBroadcastExchange : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.web_sales - : +- CometSubqueryBroadcast - : +- CometBroadcastExchange - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- ReusedSubquery +- CometBroadcastExchange +- CometFilter +- CometNativeScan parquet spark_catalog.default.date_dim -Comet accelerated 87 out of 91 eligible operators (95%). Final plan contains 1 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 81 out of 85 eligible operators (95%). Final plan contains 1 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14.native_datafusion/extended.txt index 2d20c40e4b..1b9b46662f 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14.native_datafusion/extended.txt @@ -11,11 +11,7 @@ CometNativeColumnarToRow : : :- CometProject : : : +- CometBroadcastHashJoin : : : :- CometNativeScan parquet spark_catalog.default.store_sales - : : : : +- CometSubqueryBroadcast - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- ReusedSubquery : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter @@ -23,11 +19,7 @@ CometNativeColumnarToRow : : :- CometProject : : : +- CometBroadcastHashJoin : : : :- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : +- CometSubqueryBroadcast - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- ReusedSubquery : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter @@ -35,11 +27,7 @@ CometNativeColumnarToRow : : +- CometProject : : +- CometBroadcastHashJoin : : :- CometNativeScan parquet spark_catalog.default.web_sales - : : : +- CometSubqueryBroadcast - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- ReusedSubquery : : +- CometBroadcastExchange : : +- CometProject : : +- CometFilter @@ -96,11 +84,7 @@ CometNativeColumnarToRow : : : : : : +- CometBroadcastHashJoin : : : : : : :- CometFilter : : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : : : : +- CometSubqueryBroadcast - : : : : : : : +- CometBroadcastExchange - : : : : : : : +- CometProject - : : : : : : : +- CometFilter - : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : : +- ReusedSubquery : : : : : : +- CometBroadcastExchange : : : : : : +- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.item @@ -119,11 +103,7 @@ CometNativeColumnarToRow : : : : +- CometBroadcastHashJoin : : : : :- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : : : +- CometSubqueryBroadcast - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- ReusedSubquery : : : : +- CometBroadcastExchange : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.item @@ -167,11 +147,7 @@ CometNativeColumnarToRow : : : : : +- CometBroadcastHashJoin : : : : : :- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : : : +- CometSubqueryBroadcast - : : : : : : +- CometBroadcastExchange - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : +- ReusedSubquery : : : : : +- CometBroadcastExchange : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.item @@ -190,11 +166,7 @@ CometNativeColumnarToRow : : : +- CometBroadcastHashJoin : : : :- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : : +- CometSubqueryBroadcast - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- ReusedSubquery : : : +- CometBroadcastExchange : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.item @@ -266,11 +238,7 @@ CometNativeColumnarToRow : : : : : +- CometBroadcastHashJoin : : : : : :- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : : : +- CometSubqueryBroadcast - : : : : : : +- CometBroadcastExchange - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : +- ReusedSubquery : : : : : +- CometBroadcastExchange : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.item @@ -289,11 +257,7 @@ CometNativeColumnarToRow : : : +- CometBroadcastHashJoin : : : :- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : : +- CometSubqueryBroadcast - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- ReusedSubquery : : : +- CometBroadcastExchange : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.item @@ -337,11 +301,7 @@ CometNativeColumnarToRow : : : : +- CometBroadcastHashJoin : : : : :- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : : +- CometSubqueryBroadcast - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- ReusedSubquery : : : : +- CometBroadcastExchange : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.item @@ -360,11 +320,7 @@ CometNativeColumnarToRow : : +- CometBroadcastHashJoin : : :- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : +- CometSubqueryBroadcast - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- ReusedSubquery : : +- CometBroadcastExchange : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.item @@ -382,4 +338,4 @@ CometNativeColumnarToRow : +- CometNativeScan parquet spark_catalog.default.date_dim +- CometNativeScan parquet spark_catalog.default.date_dim -Comet accelerated 354 out of 377 eligible operators (93%). Final plan contains 6 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 310 out of 333 eligible operators (93%). Final plan contains 6 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14a.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14a.native_datafusion/extended.txt index 83b545c34c..64d8fff5be 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14a.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14a.native_datafusion/extended.txt @@ -18,11 +18,7 @@ CometNativeColumnarToRow : : : :- CometProject : : : : +- CometBroadcastHashJoin : : : : :- CometNativeScan parquet spark_catalog.default.store_sales - : : : : : +- CometSubqueryBroadcast - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- ReusedSubquery : : : : +- CometBroadcastExchange : : : : +- CometProject : : : : +- CometFilter @@ -42,11 +38,7 @@ CometNativeColumnarToRow : : : +- CometProject : : : +- CometBroadcastHashJoin : : : :- CometNativeScan parquet spark_catalog.default.web_sales - : : : : +- CometSubqueryBroadcast - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- ReusedSubquery : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter @@ -98,11 +90,7 @@ CometNativeColumnarToRow : : : : : : : +- CometBroadcastHashJoin : : : : : : : :- CometFilter : : : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : : : : : +- CometSubqueryBroadcast - : : : : : : : : +- CometBroadcastExchange - : : : : : : : : +- CometProject - : : : : : : : : +- CometFilter - : : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : : : +- ReusedSubquery : : : : : : : +- CometBroadcastExchange : : : : : : : +- CometFilter : : : : : : : +- CometNativeScan parquet spark_catalog.default.item @@ -121,11 +109,7 @@ CometNativeColumnarToRow : : : : : +- CometBroadcastHashJoin : : : : : :- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : : : : +- CometSubqueryBroadcast - : : : : : : +- CometBroadcastExchange - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : +- ReusedSubquery : : : : : +- CometBroadcastExchange : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.item @@ -169,11 +153,7 @@ CometNativeColumnarToRow : : : : : : +- CometBroadcastHashJoin : : : : : : :- CometFilter : : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : : : : +- CometSubqueryBroadcast - : : : : : : : +- CometBroadcastExchange - : : : : : : : +- CometProject - : : : : : : : +- CometFilter - : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : : +- ReusedSubquery : : : : : : +- CometBroadcastExchange : : : : : : +- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.item @@ -192,11 +172,7 @@ CometNativeColumnarToRow : : : : +- CometBroadcastHashJoin : : : : :- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : : : +- CometSubqueryBroadcast - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- ReusedSubquery : : : : +- CometBroadcastExchange : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.item @@ -220,11 +196,7 @@ CometNativeColumnarToRow : : : :- CometBroadcastHashJoin : : : : :- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : : +- CometSubqueryBroadcast - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- ReusedSubquery : : : : +- CometBroadcastExchange : : : : +- CometProject : : : : +- CometBroadcastHashJoin @@ -257,11 +229,7 @@ CometNativeColumnarToRow : : : : : : : +- CometBroadcastHashJoin : : : : : : : :- CometFilter : : : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : : : : : +- CometSubqueryBroadcast - : : : : : : : : +- CometBroadcastExchange - : : : : : : : : +- CometProject - : : : : : : : : +- CometFilter - : : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : : : +- ReusedSubquery : : : : : : : +- CometBroadcastExchange : : : : : : : +- CometFilter : : : : : : : +- CometNativeScan parquet spark_catalog.default.item @@ -280,11 +248,7 @@ CometNativeColumnarToRow : : : : : +- CometBroadcastHashJoin : : : : : :- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : : : : +- CometSubqueryBroadcast - : : : : : : +- CometBroadcastExchange - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : +- ReusedSubquery : : : : : +- CometBroadcastExchange : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.item @@ -328,11 +292,7 @@ CometNativeColumnarToRow : : : : : : +- CometBroadcastHashJoin : : : : : : :- CometFilter : : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : : : : +- CometSubqueryBroadcast - : : : : : : : +- CometBroadcastExchange - : : : : : : : +- CometProject - : : : : : : : +- CometFilter - : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : : +- ReusedSubquery : : : : : : +- CometBroadcastExchange : : : : : : +- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.item @@ -351,11 +311,7 @@ CometNativeColumnarToRow : : : : +- CometBroadcastHashJoin : : : : :- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : : : +- CometSubqueryBroadcast - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- ReusedSubquery : : : : +- CometBroadcastExchange : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.item @@ -379,11 +335,7 @@ CometNativeColumnarToRow : : :- CometBroadcastHashJoin : : : :- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : : +- CometSubqueryBroadcast - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- ReusedSubquery : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometBroadcastHashJoin @@ -416,11 +368,7 @@ CometNativeColumnarToRow : : : : : : +- CometBroadcastHashJoin : : : : : : :- CometFilter : : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : : : : +- CometSubqueryBroadcast - : : : : : : : +- CometBroadcastExchange - : : : : : : : +- CometProject - : : : : : : : +- CometFilter - : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : : +- ReusedSubquery : : : : : : +- CometBroadcastExchange : : : : : : +- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.item @@ -439,11 +387,7 @@ CometNativeColumnarToRow : : : : +- CometBroadcastHashJoin : : : : :- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : : : +- CometSubqueryBroadcast - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- ReusedSubquery : : : : +- CometBroadcastExchange : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.item @@ -487,11 +431,7 @@ CometNativeColumnarToRow : : : : : +- CometBroadcastHashJoin : : : : : :- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : : : +- CometSubqueryBroadcast - : : : : : : +- CometBroadcastExchange - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : +- ReusedSubquery : : : : : +- CometBroadcastExchange : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.item @@ -510,11 +450,7 @@ CometNativeColumnarToRow : : : +- CometBroadcastHashJoin : : : :- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : : +- CometSubqueryBroadcast - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- ReusedSubquery : : : +- CometBroadcastExchange : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.item @@ -543,11 +479,7 @@ CometNativeColumnarToRow : : : :- CometProject : : : : +- CometBroadcastHashJoin : : : : :- CometNativeScan parquet spark_catalog.default.store_sales - : : : : : +- CometSubqueryBroadcast - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- ReusedSubquery : : : : +- CometBroadcastExchange : : : : +- CometProject : : : : +- CometFilter @@ -567,11 +499,7 @@ CometNativeColumnarToRow : : : +- CometProject : : : +- CometBroadcastHashJoin : : : :- CometNativeScan parquet spark_catalog.default.web_sales - : : : : +- CometSubqueryBroadcast - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- ReusedSubquery : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter @@ -623,11 +551,7 @@ CometNativeColumnarToRow : : : : : : : +- CometBroadcastHashJoin : : : : : : : :- CometFilter : : : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : : : : : +- CometSubqueryBroadcast - : : : : : : : : +- CometBroadcastExchange - : : : : : : : : +- CometProject - : : : : : : : : +- CometFilter - : : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : : : +- ReusedSubquery : : : : : : : +- CometBroadcastExchange : : : : : : : +- CometFilter : : : : : : : +- CometNativeScan parquet spark_catalog.default.item @@ -646,11 +570,7 @@ CometNativeColumnarToRow : : : : : +- CometBroadcastHashJoin : : : : : :- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : : : : +- CometSubqueryBroadcast - : : : : : : +- CometBroadcastExchange - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : +- ReusedSubquery : : : : : +- CometBroadcastExchange : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.item @@ -694,11 +614,7 @@ CometNativeColumnarToRow : : : : : : +- CometBroadcastHashJoin : : : : : : :- CometFilter : : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : : : : +- CometSubqueryBroadcast - : : : : : : : +- CometBroadcastExchange - : : : : : : : +- CometProject - : : : : : : : +- CometFilter - : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : : +- ReusedSubquery : : : : : : +- CometBroadcastExchange : : : : : : +- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.item @@ -717,11 +633,7 @@ CometNativeColumnarToRow : : : : +- CometBroadcastHashJoin : : : : :- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : : : +- CometSubqueryBroadcast - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- ReusedSubquery : : : : +- CometBroadcastExchange : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.item @@ -745,11 +657,7 @@ CometNativeColumnarToRow : : : :- CometBroadcastHashJoin : : : : :- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : : +- CometSubqueryBroadcast - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- ReusedSubquery : : : : +- CometBroadcastExchange : : : : +- CometProject : : : : +- CometBroadcastHashJoin @@ -782,11 +690,7 @@ CometNativeColumnarToRow : : : : : : : +- CometBroadcastHashJoin : : : : : : : :- CometFilter : : : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : : : : : +- CometSubqueryBroadcast - : : : : : : : : +- CometBroadcastExchange - : : : : : : : : +- CometProject - : : : : : : : : +- CometFilter - : : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : : : +- ReusedSubquery : : : : : : : +- CometBroadcastExchange : : : : : : : +- CometFilter : : : : : : : +- CometNativeScan parquet spark_catalog.default.item @@ -805,11 +709,7 @@ CometNativeColumnarToRow : : : : : +- CometBroadcastHashJoin : : : : : :- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : : : : +- CometSubqueryBroadcast - : : : : : : +- CometBroadcastExchange - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : +- ReusedSubquery : : : : : +- CometBroadcastExchange : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.item @@ -853,11 +753,7 @@ CometNativeColumnarToRow : : : : : : +- CometBroadcastHashJoin : : : : : : :- CometFilter : : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : : : : +- CometSubqueryBroadcast - : : : : : : : +- CometBroadcastExchange - : : : : : : : +- CometProject - : : : : : : : +- CometFilter - : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : : +- ReusedSubquery : : : : : : +- CometBroadcastExchange : : : : : : +- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.item @@ -876,11 +772,7 @@ CometNativeColumnarToRow : : : : +- CometBroadcastHashJoin : : : : :- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : : : +- CometSubqueryBroadcast - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- ReusedSubquery : : : : +- CometBroadcastExchange : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.item @@ -904,11 +796,7 @@ CometNativeColumnarToRow : : :- CometBroadcastHashJoin : : : :- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : : +- CometSubqueryBroadcast - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- ReusedSubquery : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometBroadcastHashJoin @@ -941,11 +829,7 @@ CometNativeColumnarToRow : : : : : : +- CometBroadcastHashJoin : : : : : : :- CometFilter : : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : : : : +- CometSubqueryBroadcast - : : : : : : : +- CometBroadcastExchange - : : : : : : : +- CometProject - : : : : : : : +- CometFilter - : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : : +- ReusedSubquery : : : : : : +- CometBroadcastExchange : : : : : : +- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.item @@ -964,11 +848,7 @@ CometNativeColumnarToRow : : : : +- CometBroadcastHashJoin : : : : :- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : : : +- CometSubqueryBroadcast - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- ReusedSubquery : : : : +- CometBroadcastExchange : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.item @@ -1012,11 +892,7 @@ CometNativeColumnarToRow : : : : : +- CometBroadcastHashJoin : : : : : :- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : : : +- CometSubqueryBroadcast - : : : : : : +- CometBroadcastExchange - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : +- ReusedSubquery : : : : : +- CometBroadcastExchange : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.item @@ -1035,11 +911,7 @@ CometNativeColumnarToRow : : : +- CometBroadcastHashJoin : : : :- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : : +- CometSubqueryBroadcast - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- ReusedSubquery : : : +- CometBroadcastExchange : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.item @@ -1068,11 +940,7 @@ CometNativeColumnarToRow : : : :- CometProject : : : : +- CometBroadcastHashJoin : : : : :- CometNativeScan parquet spark_catalog.default.store_sales - : : : : : +- CometSubqueryBroadcast - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- ReusedSubquery : : : : +- CometBroadcastExchange : : : : +- CometProject : : : : +- CometFilter @@ -1092,11 +960,7 @@ CometNativeColumnarToRow : : : +- CometProject : : : +- CometBroadcastHashJoin : : : :- CometNativeScan parquet spark_catalog.default.web_sales - : : : : +- CometSubqueryBroadcast - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- ReusedSubquery : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter @@ -1148,11 +1012,7 @@ CometNativeColumnarToRow : : : : : : : +- CometBroadcastHashJoin : : : : : : : :- CometFilter : : : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : : : : : +- CometSubqueryBroadcast - : : : : : : : : +- CometBroadcastExchange - : : : : : : : : +- CometProject - : : : : : : : : +- CometFilter - : : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : : : +- ReusedSubquery : : : : : : : +- CometBroadcastExchange : : : : : : : +- CometFilter : : : : : : : +- CometNativeScan parquet spark_catalog.default.item @@ -1171,11 +1031,7 @@ CometNativeColumnarToRow : : : : : +- CometBroadcastHashJoin : : : : : :- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : : : : +- CometSubqueryBroadcast - : : : : : : +- CometBroadcastExchange - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : +- ReusedSubquery : : : : : +- CometBroadcastExchange : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.item @@ -1219,11 +1075,7 @@ CometNativeColumnarToRow : : : : : : +- CometBroadcastHashJoin : : : : : : :- CometFilter : : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : : : : +- CometSubqueryBroadcast - : : : : : : : +- CometBroadcastExchange - : : : : : : : +- CometProject - : : : : : : : +- CometFilter - : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : : +- ReusedSubquery : : : : : : +- CometBroadcastExchange : : : : : : +- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.item @@ -1242,11 +1094,7 @@ CometNativeColumnarToRow : : : : +- CometBroadcastHashJoin : : : : :- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : : : +- CometSubqueryBroadcast - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- ReusedSubquery : : : : +- CometBroadcastExchange : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.item @@ -1270,11 +1118,7 @@ CometNativeColumnarToRow : : : :- CometBroadcastHashJoin : : : : :- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : : +- CometSubqueryBroadcast - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- ReusedSubquery : : : : +- CometBroadcastExchange : : : : +- CometProject : : : : +- CometBroadcastHashJoin @@ -1307,11 +1151,7 @@ CometNativeColumnarToRow : : : : : : : +- CometBroadcastHashJoin : : : : : : : :- CometFilter : : : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : : : : : +- CometSubqueryBroadcast - : : : : : : : : +- CometBroadcastExchange - : : : : : : : : +- CometProject - : : : : : : : : +- CometFilter - : : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : : : +- ReusedSubquery : : : : : : : +- CometBroadcastExchange : : : : : : : +- CometFilter : : : : : : : +- CometNativeScan parquet spark_catalog.default.item @@ -1330,11 +1170,7 @@ CometNativeColumnarToRow : : : : : +- CometBroadcastHashJoin : : : : : :- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : : : : +- CometSubqueryBroadcast - : : : : : : +- CometBroadcastExchange - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : +- ReusedSubquery : : : : : +- CometBroadcastExchange : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.item @@ -1378,11 +1214,7 @@ CometNativeColumnarToRow : : : : : : +- CometBroadcastHashJoin : : : : : : :- CometFilter : : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : : : : +- CometSubqueryBroadcast - : : : : : : : +- CometBroadcastExchange - : : : : : : : +- CometProject - : : : : : : : +- CometFilter - : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : : +- ReusedSubquery : : : : : : +- CometBroadcastExchange : : : : : : +- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.item @@ -1401,11 +1233,7 @@ CometNativeColumnarToRow : : : : +- CometBroadcastHashJoin : : : : :- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : : : +- CometSubqueryBroadcast - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- ReusedSubquery : : : : +- CometBroadcastExchange : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.item @@ -1429,11 +1257,7 @@ CometNativeColumnarToRow : : :- CometBroadcastHashJoin : : : :- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : : +- CometSubqueryBroadcast - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- ReusedSubquery : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometBroadcastHashJoin @@ -1466,11 +1290,7 @@ CometNativeColumnarToRow : : : : : : +- CometBroadcastHashJoin : : : : : : :- CometFilter : : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : : : : +- CometSubqueryBroadcast - : : : : : : : +- CometBroadcastExchange - : : : : : : : +- CometProject - : : : : : : : +- CometFilter - : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : : +- ReusedSubquery : : : : : : +- CometBroadcastExchange : : : : : : +- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.item @@ -1489,11 +1309,7 @@ CometNativeColumnarToRow : : : : +- CometBroadcastHashJoin : : : : :- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : : : +- CometSubqueryBroadcast - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- ReusedSubquery : : : : +- CometBroadcastExchange : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.item @@ -1537,11 +1353,7 @@ CometNativeColumnarToRow : : : : : +- CometBroadcastHashJoin : : : : : :- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : : : +- CometSubqueryBroadcast - : : : : : : +- CometBroadcastExchange - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : +- ReusedSubquery : : : : : +- CometBroadcastExchange : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.item @@ -1560,11 +1372,7 @@ CometNativeColumnarToRow : : : +- CometBroadcastHashJoin : : : :- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : : +- CometSubqueryBroadcast - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- ReusedSubquery : : : +- CometBroadcastExchange : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.item @@ -1593,11 +1401,7 @@ CometNativeColumnarToRow : : : :- CometProject : : : : +- CometBroadcastHashJoin : : : : :- CometNativeScan parquet spark_catalog.default.store_sales - : : : : : +- CometSubqueryBroadcast - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- ReusedSubquery : : : : +- CometBroadcastExchange : : : : +- CometProject : : : : +- CometFilter @@ -1617,11 +1421,7 @@ CometNativeColumnarToRow : : : +- CometProject : : : +- CometBroadcastHashJoin : : : :- CometNativeScan parquet spark_catalog.default.web_sales - : : : : +- CometSubqueryBroadcast - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- ReusedSubquery : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter @@ -1673,11 +1473,7 @@ CometNativeColumnarToRow : : : : : : : +- CometBroadcastHashJoin : : : : : : : :- CometFilter : : : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : : : : : +- CometSubqueryBroadcast - : : : : : : : : +- CometBroadcastExchange - : : : : : : : : +- CometProject - : : : : : : : : +- CometFilter - : : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : : : +- ReusedSubquery : : : : : : : +- CometBroadcastExchange : : : : : : : +- CometFilter : : : : : : : +- CometNativeScan parquet spark_catalog.default.item @@ -1696,11 +1492,7 @@ CometNativeColumnarToRow : : : : : +- CometBroadcastHashJoin : : : : : :- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : : : : +- CometSubqueryBroadcast - : : : : : : +- CometBroadcastExchange - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : +- ReusedSubquery : : : : : +- CometBroadcastExchange : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.item @@ -1744,11 +1536,7 @@ CometNativeColumnarToRow : : : : : : +- CometBroadcastHashJoin : : : : : : :- CometFilter : : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : : : : +- CometSubqueryBroadcast - : : : : : : : +- CometBroadcastExchange - : : : : : : : +- CometProject - : : : : : : : +- CometFilter - : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : : +- ReusedSubquery : : : : : : +- CometBroadcastExchange : : : : : : +- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.item @@ -1767,11 +1555,7 @@ CometNativeColumnarToRow : : : : +- CometBroadcastHashJoin : : : : :- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : : : +- CometSubqueryBroadcast - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- ReusedSubquery : : : : +- CometBroadcastExchange : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.item @@ -1795,11 +1579,7 @@ CometNativeColumnarToRow : : : :- CometBroadcastHashJoin : : : : :- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : : +- CometSubqueryBroadcast - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- ReusedSubquery : : : : +- CometBroadcastExchange : : : : +- CometProject : : : : +- CometBroadcastHashJoin @@ -1832,11 +1612,7 @@ CometNativeColumnarToRow : : : : : : : +- CometBroadcastHashJoin : : : : : : : :- CometFilter : : : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : : : : : +- CometSubqueryBroadcast - : : : : : : : : +- CometBroadcastExchange - : : : : : : : : +- CometProject - : : : : : : : : +- CometFilter - : : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : : : +- ReusedSubquery : : : : : : : +- CometBroadcastExchange : : : : : : : +- CometFilter : : : : : : : +- CometNativeScan parquet spark_catalog.default.item @@ -1855,11 +1631,7 @@ CometNativeColumnarToRow : : : : : +- CometBroadcastHashJoin : : : : : :- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : : : : +- CometSubqueryBroadcast - : : : : : : +- CometBroadcastExchange - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : +- ReusedSubquery : : : : : +- CometBroadcastExchange : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.item @@ -1903,11 +1675,7 @@ CometNativeColumnarToRow : : : : : : +- CometBroadcastHashJoin : : : : : : :- CometFilter : : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : : : : +- CometSubqueryBroadcast - : : : : : : : +- CometBroadcastExchange - : : : : : : : +- CometProject - : : : : : : : +- CometFilter - : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : : +- ReusedSubquery : : : : : : +- CometBroadcastExchange : : : : : : +- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.item @@ -1926,11 +1694,7 @@ CometNativeColumnarToRow : : : : +- CometBroadcastHashJoin : : : : :- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : : : +- CometSubqueryBroadcast - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- ReusedSubquery : : : : +- CometBroadcastExchange : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.item @@ -1954,11 +1718,7 @@ CometNativeColumnarToRow : : :- CometBroadcastHashJoin : : : :- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : : +- CometSubqueryBroadcast - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- ReusedSubquery : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometBroadcastHashJoin @@ -1991,11 +1751,7 @@ CometNativeColumnarToRow : : : : : : +- CometBroadcastHashJoin : : : : : : :- CometFilter : : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : : : : +- CometSubqueryBroadcast - : : : : : : : +- CometBroadcastExchange - : : : : : : : +- CometProject - : : : : : : : +- CometFilter - : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : : +- ReusedSubquery : : : : : : +- CometBroadcastExchange : : : : : : +- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.item @@ -2014,11 +1770,7 @@ CometNativeColumnarToRow : : : : +- CometBroadcastHashJoin : : : : :- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : : : +- CometSubqueryBroadcast - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- ReusedSubquery : : : : +- CometBroadcastExchange : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.item @@ -2062,11 +1814,7 @@ CometNativeColumnarToRow : : : : : +- CometBroadcastHashJoin : : : : : :- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : : : +- CometSubqueryBroadcast - : : : : : : +- CometBroadcastExchange - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : +- ReusedSubquery : : : : : +- CometBroadcastExchange : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.item @@ -2085,11 +1833,7 @@ CometNativeColumnarToRow : : : +- CometBroadcastHashJoin : : : :- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : : +- CometSubqueryBroadcast - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- ReusedSubquery : : : +- CometBroadcastExchange : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.item @@ -2118,11 +1862,7 @@ CometNativeColumnarToRow : : :- CometProject : : : +- CometBroadcastHashJoin : : : :- CometNativeScan parquet spark_catalog.default.store_sales - : : : : +- CometSubqueryBroadcast - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- ReusedSubquery : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter @@ -2142,11 +1882,7 @@ CometNativeColumnarToRow : : +- CometProject : : +- CometBroadcastHashJoin : : :- CometNativeScan parquet spark_catalog.default.web_sales - : : : +- CometSubqueryBroadcast - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- ReusedSubquery : : +- CometBroadcastExchange : : +- CometProject : : +- CometFilter @@ -2198,11 +1934,7 @@ CometNativeColumnarToRow : : : : : : +- CometBroadcastHashJoin : : : : : : :- CometFilter : : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : : : : +- CometSubqueryBroadcast - : : : : : : : +- CometBroadcastExchange - : : : : : : : +- CometProject - : : : : : : : +- CometFilter - : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : : +- ReusedSubquery : : : : : : +- CometBroadcastExchange : : : : : : +- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.item @@ -2221,11 +1953,7 @@ CometNativeColumnarToRow : : : : +- CometBroadcastHashJoin : : : : :- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : : : +- CometSubqueryBroadcast - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- ReusedSubquery : : : : +- CometBroadcastExchange : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.item @@ -2269,11 +1997,7 @@ CometNativeColumnarToRow : : : : : +- CometBroadcastHashJoin : : : : : :- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : : : +- CometSubqueryBroadcast - : : : : : : +- CometBroadcastExchange - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : +- ReusedSubquery : : : : : +- CometBroadcastExchange : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.item @@ -2292,11 +2016,7 @@ CometNativeColumnarToRow : : : +- CometBroadcastHashJoin : : : :- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : : +- CometSubqueryBroadcast - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- ReusedSubquery : : : +- CometBroadcastExchange : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.item @@ -2320,11 +2040,7 @@ CometNativeColumnarToRow : : :- CometBroadcastHashJoin : : : :- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : +- CometSubqueryBroadcast - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- ReusedSubquery : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometBroadcastHashJoin @@ -2357,11 +2073,7 @@ CometNativeColumnarToRow : : : : : : +- CometBroadcastHashJoin : : : : : : :- CometFilter : : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : : : : +- CometSubqueryBroadcast - : : : : : : : +- CometBroadcastExchange - : : : : : : : +- CometProject - : : : : : : : +- CometFilter - : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : : +- ReusedSubquery : : : : : : +- CometBroadcastExchange : : : : : : +- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.item @@ -2380,11 +2092,7 @@ CometNativeColumnarToRow : : : : +- CometBroadcastHashJoin : : : : :- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : : : +- CometSubqueryBroadcast - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- ReusedSubquery : : : : +- CometBroadcastExchange : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.item @@ -2428,11 +2136,7 @@ CometNativeColumnarToRow : : : : : +- CometBroadcastHashJoin : : : : : :- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : : : +- CometSubqueryBroadcast - : : : : : : +- CometBroadcastExchange - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : +- ReusedSubquery : : : : : +- CometBroadcastExchange : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.item @@ -2451,11 +2155,7 @@ CometNativeColumnarToRow : : : +- CometBroadcastHashJoin : : : :- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : : +- CometSubqueryBroadcast - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- ReusedSubquery : : : +- CometBroadcastExchange : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.item @@ -2479,11 +2179,7 @@ CometNativeColumnarToRow : :- CometBroadcastHashJoin : : :- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : +- CometSubqueryBroadcast - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- ReusedSubquery : : +- CometBroadcastExchange : : +- CometProject : : +- CometBroadcastHashJoin @@ -2516,11 +2212,7 @@ CometNativeColumnarToRow : : : : : +- CometBroadcastHashJoin : : : : : :- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : : : +- CometSubqueryBroadcast - : : : : : : +- CometBroadcastExchange - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : +- ReusedSubquery : : : : : +- CometBroadcastExchange : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.item @@ -2539,11 +2231,7 @@ CometNativeColumnarToRow : : : +- CometBroadcastHashJoin : : : :- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : : +- CometSubqueryBroadcast - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- ReusedSubquery : : : +- CometBroadcastExchange : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.item @@ -2587,11 +2275,7 @@ CometNativeColumnarToRow : : : : +- CometBroadcastHashJoin : : : : :- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : : +- CometSubqueryBroadcast - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- ReusedSubquery : : : : +- CometBroadcastExchange : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.item @@ -2610,11 +2294,7 @@ CometNativeColumnarToRow : : +- CometBroadcastHashJoin : : :- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : +- CometSubqueryBroadcast - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- ReusedSubquery : : +- CometBroadcastExchange : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.item @@ -2627,4 +2307,4 @@ CometNativeColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.date_dim -Comet accelerated 2487 out of 2622 eligible operators (94%). Final plan contains 6 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 2167 out of 2302 eligible operators (94%). Final plan contains 6 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q18a.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q18a.native_datafusion/extended.txt index b34ebfdf84..75f132f6fb 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q18a.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q18a.native_datafusion/extended.txt @@ -63,11 +63,7 @@ CometNativeColumnarToRow : : : : : : +- CometBroadcastHashJoin : : : : : : :- CometFilter : : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : : : : +- CometSubqueryBroadcast - : : : : : : : +- CometBroadcastExchange - : : : : : : : +- CometProject - : : : : : : : +- CometFilter - : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : : +- ReusedSubquery : : : : : : +- CometBroadcastExchange : : : : : : +- CometProject : : : : : : +- CometFilter @@ -108,11 +104,7 @@ CometNativeColumnarToRow : : : : : : +- CometBroadcastHashJoin : : : : : : :- CometFilter : : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : : : : +- CometSubqueryBroadcast - : : : : : : : +- CometBroadcastExchange - : : : : : : : +- CometProject - : : : : : : : +- CometFilter - : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : : +- ReusedSubquery : : : : : : +- CometBroadcastExchange : : : : : : +- CometProject : : : : : : +- CometFilter @@ -153,11 +145,7 @@ CometNativeColumnarToRow : : : : : : +- CometBroadcastHashJoin : : : : : : :- CometFilter : : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : : : : +- CometSubqueryBroadcast - : : : : : : : +- CometBroadcastExchange - : : : : : : : +- CometProject - : : : : : : : +- CometFilter - : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : : +- ReusedSubquery : : : : : : +- CometBroadcastExchange : : : : : : +- CometProject : : : : : : +- CometFilter @@ -198,11 +186,7 @@ CometNativeColumnarToRow : : : : : +- CometBroadcastHashJoin : : : : : :- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : : : +- CometSubqueryBroadcast - : : : : : : +- CometBroadcastExchange - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : +- ReusedSubquery : : : : : +- CometBroadcastExchange : : : : : +- CometProject : : : : : +- CometFilter @@ -226,4 +210,4 @@ CometNativeColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.item -Comet accelerated 221 out of 226 eligible operators (97%). Final plan contains 1 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 205 out of 210 eligible operators (97%). Final plan contains 1 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q27a.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q27a.native_datafusion/extended.txt index 9c4497cbd3..70614792ba 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q27a.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q27a.native_datafusion/extended.txt @@ -48,11 +48,7 @@ CometNativeColumnarToRow : : : : +- CometBroadcastHashJoin : : : : :- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : : : +- CometSubqueryBroadcast - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- ReusedSubquery : : : : +- CometBroadcastExchange : : : : +- CometProject : : : : +- CometFilter @@ -82,11 +78,7 @@ CometNativeColumnarToRow : : : +- CometBroadcastHashJoin : : : :- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : : +- CometSubqueryBroadcast - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- ReusedSubquery : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter @@ -103,4 +95,4 @@ CometNativeColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.item -Comet accelerated 100 out of 103 eligible operators (97%). Final plan contains 1 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 92 out of 95 eligible operators (96%). Final plan contains 1 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q35.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q35.native_datafusion/extended.txt index 2e79f6f464..b7c8209556 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q35.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q35.native_datafusion/extended.txt @@ -33,11 +33,7 @@ TakeOrderedAndProject : : : +- CometProject : : : +- CometBroadcastHashJoin : : : :- CometNativeScan parquet spark_catalog.default.web_sales - : : : : +- CometSubqueryBroadcast - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- ReusedSubquery : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter @@ -47,11 +43,7 @@ TakeOrderedAndProject : : +- CometProject : : +- CometBroadcastHashJoin : : :- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : +- CometSubqueryBroadcast - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- ReusedSubquery : : +- CometBroadcastExchange : : +- CometProject : : +- CometFilter @@ -67,4 +59,4 @@ TakeOrderedAndProject +- CometFilter +- CometNativeScan parquet spark_catalog.default.customer_demographics -Comet accelerated 44 out of 62 eligible operators (70%). Final plan contains 6 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 36 out of 54 eligible operators (66%). Final plan contains 6 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q35a.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q35a.native_datafusion/extended.txt index 173c450c71..f27de2fdbb 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q35a.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q35a.native_datafusion/extended.txt @@ -30,11 +30,7 @@ CometNativeColumnarToRow : : :- CometProject : : : +- CometBroadcastHashJoin : : : :- CometNativeScan parquet spark_catalog.default.web_sales - : : : : +- CometSubqueryBroadcast - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- ReusedSubquery : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter @@ -42,11 +38,7 @@ CometNativeColumnarToRow : : +- CometProject : : +- CometBroadcastHashJoin : : :- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : +- CometSubqueryBroadcast - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- ReusedSubquery : : +- CometBroadcastExchange : : +- CometProject : : +- CometFilter @@ -60,4 +52,4 @@ CometNativeColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.customer_demographics -Comet accelerated 57 out of 60 eligible operators (95%). Final plan contains 1 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 49 out of 52 eligible operators (94%). Final plan contains 1 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q49.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q49.native_datafusion/extended.txt index f2ae44fb86..cd6c39e4f3 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q49.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q49.native_datafusion/extended.txt @@ -54,11 +54,7 @@ CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : +- CometSubqueryBroadcast - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- ReusedSubquery : : +- CometProject : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.catalog_returns @@ -85,11 +81,7 @@ CometNativeColumnarToRow : : +- CometProject : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : +- CometSubqueryBroadcast - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- ReusedSubquery : +- CometProject : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.store_returns @@ -98,4 +90,4 @@ CometNativeColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.date_dim -Comet accelerated 75 out of 95 eligible operators (78%). Final plan contains 4 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 67 out of 87 eligible operators (77%). Final plan contains 4 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q51a.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q51a.native_datafusion/extended.txt index a3400c5ecf..c8010c1286 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q51a.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q51a.native_datafusion/extended.txt @@ -82,11 +82,7 @@ TakeOrderedAndProject : : +- CometBroadcastHashJoin : : :- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : +- CometSubqueryBroadcast - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- ReusedSubquery : : +- CometBroadcastExchange : : +- CometProject : : +- CometFilter @@ -104,11 +100,7 @@ TakeOrderedAndProject : +- CometBroadcastHashJoin : :- CometFilter : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : +- CometSubqueryBroadcast - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- ReusedSubquery : +- CometBroadcastExchange : +- CometProject : +- CometFilter @@ -193,11 +185,7 @@ TakeOrderedAndProject : +- CometBroadcastHashJoin : :- CometFilter : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : +- CometSubqueryBroadcast - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- ReusedSubquery : +- CometBroadcastExchange : +- CometProject : +- CometFilter @@ -215,14 +203,10 @@ TakeOrderedAndProject +- CometBroadcastHashJoin :- CometFilter : +- CometNativeScan parquet spark_catalog.default.store_sales - : +- CometSubqueryBroadcast - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- ReusedSubquery +- CometBroadcastExchange +- CometProject +- CometFilter +- CometNativeScan parquet spark_catalog.default.date_dim -Comet accelerated 158 out of 212 eligible operators (74%). Final plan contains 14 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 142 out of 196 eligible operators (72%). Final plan contains 14 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q5a.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q5a.native_datafusion/extended.txt index b472530c10..8d61d1e1ad 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q5a.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q5a.native_datafusion/extended.txt @@ -27,11 +27,7 @@ CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.store_returns - : : : : +- CometSubqueryBroadcast - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- ReusedSubquery : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter @@ -51,19 +47,11 @@ CometNativeColumnarToRow : : : : :- CometProject : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : : +- CometSubqueryBroadcast - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- ReusedSubquery : : : : +- CometProject : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.catalog_returns - : : : : +- CometSubqueryBroadcast - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- ReusedSubquery : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter @@ -83,20 +71,12 @@ CometNativeColumnarToRow : : : :- CometProject : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : : +- CometSubqueryBroadcast - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- ReusedSubquery : : : +- CometProject : : : +- CometBroadcastHashJoin : : : :- CometBroadcastExchange : : : : +- CometNativeScan parquet spark_catalog.default.web_returns - : : : : +- CometSubqueryBroadcast - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- ReusedSubquery : : : +- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.web_sales @@ -134,11 +114,7 @@ CometNativeColumnarToRow : : : : +- CometProject : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.store_returns - : : : : +- CometSubqueryBroadcast - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- ReusedSubquery : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter @@ -158,19 +134,11 @@ CometNativeColumnarToRow : : : : :- CometProject : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : : +- CometSubqueryBroadcast - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- ReusedSubquery : : : : +- CometProject : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.catalog_returns - : : : : +- CometSubqueryBroadcast - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- ReusedSubquery : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter @@ -190,20 +158,12 @@ CometNativeColumnarToRow : : : :- CometProject : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : : +- CometSubqueryBroadcast - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- ReusedSubquery : : : +- CometProject : : : +- CometBroadcastHashJoin : : : :- CometBroadcastExchange : : : : +- CometNativeScan parquet spark_catalog.default.web_returns - : : : : +- CometSubqueryBroadcast - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- ReusedSubquery : : : +- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.web_sales @@ -241,11 +201,7 @@ CometNativeColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.store_returns - : : : +- CometSubqueryBroadcast - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- ReusedSubquery : : +- CometBroadcastExchange : : +- CometProject : : +- CometFilter @@ -265,19 +221,11 @@ CometNativeColumnarToRow : : : :- CometProject : : : : +- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : +- CometSubqueryBroadcast - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- ReusedSubquery : : : +- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.catalog_returns - : : : +- CometSubqueryBroadcast - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- ReusedSubquery : : +- CometBroadcastExchange : : +- CometProject : : +- CometFilter @@ -297,20 +245,12 @@ CometNativeColumnarToRow : : :- CometProject : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : +- CometSubqueryBroadcast - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- ReusedSubquery : : +- CometProject : : +- CometBroadcastHashJoin : : :- CometBroadcastExchange : : : +- CometNativeScan parquet spark_catalog.default.web_returns - : : : +- CometSubqueryBroadcast - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- ReusedSubquery : : +- CometProject : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.web_sales @@ -323,4 +263,4 @@ CometNativeColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.web_site -Comet accelerated 305 out of 323 eligible operators (94%). Final plan contains 1 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 245 out of 263 eligible operators (93%). Final plan contains 1 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q70a.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q70a.native_datafusion/extended.txt index 5e1387f550..f2022d6e5a 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q70a.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q70a.native_datafusion/extended.txt @@ -49,11 +49,7 @@ TakeOrderedAndProject : : +- CometBroadcastHashJoin : : :- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : +- CometSubqueryBroadcast - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- ReusedSubquery : : +- CometBroadcastExchange : : +- CometProject : : +- CometFilter @@ -107,11 +103,7 @@ TakeOrderedAndProject : : +- CometBroadcastHashJoin : : :- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : +- CometSubqueryBroadcast - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- ReusedSubquery : : +- CometBroadcastExchange : : +- CometProject : : +- CometFilter @@ -165,11 +157,7 @@ TakeOrderedAndProject : +- CometBroadcastHashJoin : :- CometFilter : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : +- CometSubqueryBroadcast - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- ReusedSubquery : +- CometBroadcastExchange : +- CometProject : +- CometFilter @@ -179,4 +167,4 @@ TakeOrderedAndProject +- CometFilter +- CometNativeScan parquet spark_catalog.default.date_dim -Comet accelerated 117 out of 165 eligible operators (70%). Final plan contains 15 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 105 out of 153 eligible operators (68%). Final plan contains 15 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q74.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q74.native_datafusion/extended.txt index 7768138645..cfe634568f 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q74.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q74.native_datafusion/extended.txt @@ -62,10 +62,7 @@ CometNativeColumnarToRow : : +- CometBroadcastExchange : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : +- CometSubqueryBroadcast - : : +- CometBroadcastExchange - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- ReusedSubquery : +- CometBroadcastExchange : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.date_dim @@ -83,12 +80,9 @@ CometNativeColumnarToRow : +- CometBroadcastExchange : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.web_sales - : +- CometSubqueryBroadcast - : +- CometBroadcastExchange - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- ReusedSubquery +- CometBroadcastExchange +- CometFilter +- CometNativeScan parquet spark_catalog.default.date_dim -Comet accelerated 87 out of 91 eligible operators (95%). Final plan contains 1 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 81 out of 85 eligible operators (95%). Final plan contains 1 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q75.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q75.native_datafusion/extended.txt index ee7c0af216..bccd23b141 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q75.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q75.native_datafusion/extended.txt @@ -48,10 +48,7 @@ CometNativeColumnarToRow : : : : +- CometBroadcastHashJoin : : : : :- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : : : +- CometSubqueryBroadcast - : : : : : +- CometBroadcastExchange - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- ReusedSubquery : : : : +- CometBroadcastExchange : : : : +- CometProject : : : : +- CometFilter @@ -74,10 +71,7 @@ CometNativeColumnarToRow : : : +- CometBroadcastHashJoin : : : :- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : : +- CometSubqueryBroadcast - : : : : +- CometBroadcastExchange - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- ReusedSubquery : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter @@ -136,10 +130,7 @@ CometNativeColumnarToRow : : : +- CometBroadcastHashJoin : : : :- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : : +- CometSubqueryBroadcast - : : : : +- CometBroadcastExchange - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- ReusedSubquery : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter @@ -162,10 +153,7 @@ CometNativeColumnarToRow : : +- CometBroadcastHashJoin : : :- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : +- CometSubqueryBroadcast - : : : +- CometBroadcastExchange - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- ReusedSubquery : : +- CometBroadcastExchange : : +- CometProject : : +- CometFilter @@ -179,4 +167,4 @@ CometNativeColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.web_returns -Comet accelerated 173 out of 179 eligible operators (96%). Final plan contains 1 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 161 out of 167 eligible operators (96%). Final plan contains 1 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q77a.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q77a.native_datafusion/extended.txt index 6621cfd8f8..95b315d258 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q77a.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q77a.native_datafusion/extended.txt @@ -43,11 +43,7 @@ CometNativeColumnarToRow : : : +- CometBroadcastHashJoin : : : :- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.store_returns - : : : : +- CometSubqueryBroadcast - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- ReusedSubquery : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter @@ -65,11 +61,7 @@ CometNativeColumnarToRow : : : +- CometProject : : : +- CometBroadcastHashJoin : : : :- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : +- CometSubqueryBroadcast - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- ReusedSubquery : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter @@ -81,11 +73,7 @@ CometNativeColumnarToRow : : +- CometProject : : +- CometBroadcastHashJoin : : :- CometNativeScan parquet spark_catalog.default.catalog_returns - : : : +- CometSubqueryBroadcast - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- ReusedSubquery : : +- CometBroadcastExchange : : +- CometProject : : +- CometFilter @@ -102,11 +90,7 @@ CometNativeColumnarToRow : : : +- CometBroadcastHashJoin : : : :- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : : +- CometSubqueryBroadcast - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- ReusedSubquery : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter @@ -124,11 +108,7 @@ CometNativeColumnarToRow : : +- CometBroadcastHashJoin : : :- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.web_returns - : : : +- CometSubqueryBroadcast - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- ReusedSubquery : : +- CometBroadcastExchange : : +- CometProject : : +- CometFilter @@ -179,11 +159,7 @@ CometNativeColumnarToRow : : : +- CometBroadcastHashJoin : : : :- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.store_returns - : : : : +- CometSubqueryBroadcast - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- ReusedSubquery : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter @@ -201,11 +177,7 @@ CometNativeColumnarToRow : : : +- CometProject : : : +- CometBroadcastHashJoin : : : :- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : +- CometSubqueryBroadcast - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- ReusedSubquery : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter @@ -217,11 +189,7 @@ CometNativeColumnarToRow : : +- CometProject : : +- CometBroadcastHashJoin : : :- CometNativeScan parquet spark_catalog.default.catalog_returns - : : : +- CometSubqueryBroadcast - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- ReusedSubquery : : +- CometBroadcastExchange : : +- CometProject : : +- CometFilter @@ -238,11 +206,7 @@ CometNativeColumnarToRow : : : +- CometBroadcastHashJoin : : : :- CometFilter : : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : : +- CometSubqueryBroadcast - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- ReusedSubquery : : : +- CometBroadcastExchange : : : +- CometProject : : : +- CometFilter @@ -260,11 +224,7 @@ CometNativeColumnarToRow : : +- CometBroadcastHashJoin : : :- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.web_returns - : : : +- CometSubqueryBroadcast - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- ReusedSubquery : : +- CometBroadcastExchange : : +- CometProject : : +- CometFilter @@ -315,11 +275,7 @@ CometNativeColumnarToRow : : +- CometBroadcastHashJoin : : :- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.store_returns - : : : +- CometSubqueryBroadcast - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- ReusedSubquery : : +- CometBroadcastExchange : : +- CometProject : : +- CometFilter @@ -337,11 +293,7 @@ CometNativeColumnarToRow : : +- CometProject : : +- CometBroadcastHashJoin : : :- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : +- CometSubqueryBroadcast - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- ReusedSubquery : : +- CometBroadcastExchange : : +- CometProject : : +- CometFilter @@ -353,11 +305,7 @@ CometNativeColumnarToRow : +- CometProject : +- CometBroadcastHashJoin : :- CometNativeScan parquet spark_catalog.default.catalog_returns - : : +- CometSubqueryBroadcast - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- ReusedSubquery : +- CometBroadcastExchange : +- CometProject : +- CometFilter @@ -374,11 +322,7 @@ CometNativeColumnarToRow : : +- CometBroadcastHashJoin : : :- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : +- CometSubqueryBroadcast - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- ReusedSubquery : : +- CometBroadcastExchange : : +- CometProject : : +- CometFilter @@ -396,11 +340,7 @@ CometNativeColumnarToRow : +- CometBroadcastHashJoin : :- CometFilter : : +- CometNativeScan parquet spark_catalog.default.web_returns - : : +- CometSubqueryBroadcast - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- ReusedSubquery : +- CometBroadcastExchange : +- CometProject : +- CometFilter @@ -409,4 +349,4 @@ CometNativeColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.web_page -Comet accelerated 350 out of 392 eligible operators (89%). Final plan contains 18 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 290 out of 332 eligible operators (87%). Final plan contains 18 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q78.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q78.native_datafusion/extended.txt index 8426b9653c..d29dbc13e5 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q78.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q78.native_datafusion/extended.txt @@ -43,10 +43,7 @@ TakeOrderedAndProject : : : +- CometExchange : : : +- CometFilter : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : +- CometSubqueryBroadcast - : : : +- CometBroadcastExchange - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- ReusedSubquery : : +- CometSort : : +- CometExchange : : +- CometProject @@ -69,10 +66,7 @@ TakeOrderedAndProject : : +- CometExchange : : +- CometFilter : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : +- CometSubqueryBroadcast - : : +- CometBroadcastExchange - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- ReusedSubquery : +- CometSort : +- CometExchange : +- CometProject @@ -82,4 +76,4 @@ TakeOrderedAndProject +- CometFilter +- CometNativeScan parquet spark_catalog.default.date_dim -Comet accelerated 77 out of 82 eligible operators (93%). Final plan contains 1 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 71 out of 76 eligible operators (93%). Final plan contains 1 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q80a.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q80a.native_datafusion/extended.txt index dced05333a..10e0803953 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q80a.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q80a.native_datafusion/extended.txt @@ -68,11 +68,7 @@ CometNativeColumnarToRow : : : : : : : +- CometExchange : : : : : : : +- CometFilter : : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : : : : +- CometSubqueryBroadcast - : : : : : : : +- CometBroadcastExchange - : : : : : : : +- CometProject - : : : : : : : +- CometFilter - : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : : +- ReusedSubquery : : : : : : +- CometSort : : : : : : +- CometExchange : : : : : : +- CometProject @@ -111,11 +107,7 @@ CometNativeColumnarToRow : : : : : : +- CometExchange : : : : : : +- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : : : : +- CometSubqueryBroadcast - : : : : : : +- CometBroadcastExchange - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : +- ReusedSubquery : : : : : +- CometSort : : : : : +- CometExchange : : : : : +- CometProject @@ -204,11 +196,7 @@ CometNativeColumnarToRow : : : : : : : +- CometExchange : : : : : : : +- CometFilter : : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : : : : +- CometSubqueryBroadcast - : : : : : : : +- CometBroadcastExchange - : : : : : : : +- CometProject - : : : : : : : +- CometFilter - : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : : +- ReusedSubquery : : : : : : +- CometSort : : : : : : +- CometExchange : : : : : : +- CometProject @@ -247,11 +235,7 @@ CometNativeColumnarToRow : : : : : : +- CometExchange : : : : : : +- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : : : : +- CometSubqueryBroadcast - : : : : : : +- CometBroadcastExchange - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : +- ReusedSubquery : : : : : +- CometSort : : : : : +- CometExchange : : : : : +- CometProject @@ -340,11 +324,7 @@ CometNativeColumnarToRow : : : : : : +- CometExchange : : : : : : +- CometFilter : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : : : +- CometSubqueryBroadcast - : : : : : : +- CometBroadcastExchange - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : +- ReusedSubquery : : : : : +- CometSort : : : : : +- CometExchange : : : : : +- CometProject @@ -383,11 +363,7 @@ CometNativeColumnarToRow : : : : : +- CometExchange : : : : : +- CometFilter : : : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : : : +- CometSubqueryBroadcast - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : +- ReusedSubquery : : : : +- CometSort : : : : +- CometExchange : : : : +- CometProject @@ -410,4 +386,4 @@ CometNativeColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.promotion -Comet accelerated 401 out of 410 eligible operators (97%). Final plan contains 1 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 377 out of 386 eligible operators (97%). Final plan contains 1 transitions between Spark and Comet. \ No newline at end of file