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..4dcdc89410 100644 --- a/spark/src/main/scala/org/apache/comet/rules/CometExecRule.scala +++ b/spark/src/main/scala/org/apache/comet/rules/CometExecRule.scala @@ -88,6 +88,13 @@ object CometExecRule { val allExecs: Map[Class[_ <: SparkPlan], CometOperatorSerde[_]] = nativeExecs ++ sinks + /** + * Tag set on a `ShuffleExchangeExec` that should be left as a plain Spark shuffle rather than + * wrapped in `CometShuffleExchangeExec`. See `tagRedundantColumnarShuffle`. + */ + val SKIP_COMET_SHUFFLE_TAG: org.apache.spark.sql.catalyst.trees.TreeNodeTag[Unit] = + org.apache.spark.sql.catalyst.trees.TreeNodeTag[Unit]("comet.skipCometShuffle") + } /** @@ -97,19 +104,66 @@ case class CometExecRule(session: SparkSession) extends Rule[SparkPlan] { private lazy val showTransformations = CometConf.COMET_EXPLAIN_TRANSFORMATIONS.get() + /** + * Revert any `CometShuffleExchangeExec` with `CometColumnarShuffle` that is sandwiched between + * two non-Comet `HashAggregateExec` / `ObjectHashAggregateExec` operators back to the original + * Spark `ShuffleExchangeExec`. This is the partial-final-aggregate pattern where Comet couldn't + * convert either aggregate; keeping a columnar shuffle between them only adds + * row->arrow->shuffle->arrow->row conversion overhead with no Comet consumer on either side. + * See https://github.com/apache/datafusion-comet/issues/4004. + * + * The match is intentionally narrow (both sides must be row-based aggregates that remained JVM + * after the main transform pass). Running the revert post-transform means we only fire when the + * main conversion already decided to keep both aggregates JVM - we never create the dangerous + * mixed mode where a Comet partial feeds a JVM final (see issue #1389). + * + * Also tag the reverted shuffle so AQE stage-isolated re-planning does not convert it back to a + * Comet shuffle when the outer aggregate context is no longer visible. + */ + private def revertRedundantColumnarShuffle(plan: SparkPlan): SparkPlan = { + def isAggregate(p: SparkPlan): Boolean = + p.isInstanceOf[HashAggregateExec] || p.isInstanceOf[ObjectHashAggregateExec] + + def isRedundantShuffle(child: SparkPlan): Boolean = child match { + case s: CometShuffleExchangeExec => + s.shuffleType == CometColumnarShuffle && isAggregate(s.child) + case _ => false + } + + plan.transform { + case op if isAggregate(op) && op.children.exists(isRedundantShuffle) => + val newChildren = op.children.map { + case s: CometShuffleExchangeExec + if s.shuffleType == CometColumnarShuffle && isAggregate(s.child) => + val reverted = + s.originalPlan.withNewChildren(Seq(s.child)).asInstanceOf[ShuffleExchangeExec] + reverted.setTagValue(CometExecRule.SKIP_COMET_SHUFFLE_TAG, ()) + reverted + case other => other + } + op.withNewChildren(newChildren) + } + } + + private def shouldSkipCometShuffle(s: ShuffleExchangeExec): Boolean = + s.getTagValue(CometExecRule.SKIP_COMET_SHUFFLE_TAG).isDefined + private def applyCometShuffle(plan: SparkPlan): SparkPlan = { - plan.transformUp { case s: ShuffleExchangeExec => - CometShuffleExchangeExec.shuffleSupported(s) match { - case Some(CometNativeShuffle) => - // Switch to use Decimal128 regardless of precision, since Arrow native execution - // doesn't support Decimal32 and Decimal64 yet. - conf.setConfString(CometConf.COMET_USE_DECIMAL_128.key, "true") - CometShuffleExchangeExec(s, shuffleType = CometNativeShuffle) - case Some(CometColumnarShuffle) => - CometShuffleExchangeExec(s, shuffleType = CometColumnarShuffle) - case None => - s - } + plan.transformUp { + case s: ShuffleExchangeExec if shouldSkipCometShuffle(s) => + s + case s: ShuffleExchangeExec => + CometShuffleExchangeExec.shuffleSupported(s) match { + case Some(CometNativeShuffle) => + // Switch to use Decimal128 regardless of precision, since Arrow native execution + // doesn't support Decimal32 and Decimal64 yet. + conf.setConfString(CometConf.COMET_USE_DECIMAL_128.key, "true") + CometShuffleExchangeExec(s, shuffleType = CometNativeShuffle) + case Some(CometColumnarShuffle) => + CometShuffleExchangeExec(s, shuffleType = CometColumnarShuffle) + case None => + s + } } } @@ -258,6 +312,9 @@ case class CometExecRule(session: SparkSession) extends Rule[SparkPlan] { case s @ ShuffleQueryStageExec(_, ReusedExchangeExec(_, _: CometShuffleExchangeExec), _) => convertToComet(s, CometExchangeSink).getOrElse(s) + case s: ShuffleExchangeExec if shouldSkipCometShuffle(s) => + s + case s: ShuffleExchangeExec => convertToComet(s, CometShuffleExchangeExec).getOrElse(s) @@ -409,6 +466,11 @@ case class CometExecRule(session: SparkSession) extends Rule[SparkPlan] { case CometScanWrapper(_, s) => s } + // Revert CometColumnarShuffle to Spark's ShuffleExchangeExec when sandwiched between two + // non-Comet HashAggregate/ObjectHashAggregate operators that remained JVM after the main + // transform pass. See https://github.com/apache/datafusion-comet/issues/4004. + newPlan = revertRedundantColumnarShuffle(newPlan) + // Set up logical links newPlan = newPlan.transform { case op: CometExec => 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..a3434fe3ea 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 @@ -7,46 +7,43 @@ TakeOrderedAndProject : : +- 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 + : : : +- Exchange + : : : +- 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 + : : +- Exchange + : : +- HashAggregate : : +- 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 + : : +- Exchange + : : +- 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 @@ -58,4 +55,4 @@ TakeOrderedAndProject +- 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 15 out of 49 eligible operators (30%). 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/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..c8a9740a02 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 @@ -1,67 +1,66 @@ TakeOrderedAndProject +- HashAggregate - +- CometNativeColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- 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 - : : : : +- 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 - : : : +- 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 - : : +- 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 + +- Exchange + +- HashAggregate + +- Project + +- BroadcastHashJoin + :- Project + : +- BroadcastHashJoin + : :- 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 + : : : : +- 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 + : : : +- 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 + : : +- 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 -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 20 out of 54 eligible operators (37%). 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/q10.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q10.native_iceberg_compat/extended.txt index c5125b1861..72960fc267 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q10.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q10.native_iceberg_compat/extended.txt @@ -1,63 +1,62 @@ TakeOrderedAndProject +- HashAggregate - +- CometNativeColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- Project - : : +- Filter - : : +- BroadcastHashJoin - : : :- BroadcastHashJoin [COMET: Unsupported join type ExistenceJoin(exists#1)] - : : : :- CometNativeColumnarToRow - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : +- ReusedSubquery - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : +- ReusedSubquery - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- CometNativeColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address - +- BroadcastExchange - +- CometNativeColumnarToRow - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics + +- Exchange + +- HashAggregate + +- Project + +- BroadcastHashJoin + :- Project + : +- BroadcastHashJoin + : :- Project + : : +- Filter + : : +- BroadcastHashJoin + : : :- BroadcastHashJoin [COMET: Unsupported join type ExistenceJoin(exists#1)] + : : : :- CometNativeColumnarToRow + : : : : +- CometBroadcastHashJoin + : : : : :- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometNativeColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : +- BroadcastExchange + : : : +- CometNativeColumnarToRow + : : : +- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales + : : : : +- ReusedSubquery + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- BroadcastExchange + : : +- CometNativeColumnarToRow + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales + : : : +- ReusedSubquery + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- BroadcastExchange + : +- CometNativeColumnarToRow + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address + +- BroadcastExchange + +- CometNativeColumnarToRow + +- CometProject + +- CometFilter + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics -Comet accelerated 35 out of 54 eligible operators (64%). Final plan contains 7 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 34 out of 54 eligible operators (62%). 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 47aa9517cf..a38205d189 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 @@ -7,100 +7,96 @@ TakeOrderedAndProject : : +- 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 + : : : +- Exchange + : : : +- 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 + : : +- Exchange + : : +- 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 + : +- Exchange + : +- 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 + +- Exchange + +- 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 -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 24 out of 86 eligible operators (27%). 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-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..79e4955126 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 @@ -5,31 +5,30 @@ TakeOrderedAndProject +- 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 + +- Exchange + +- 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 -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 11 out of 27 eligible operators (40%). 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/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..886c7fc5fe 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,47 @@ 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 ++- Exchange + +- 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 -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 16 out of 38 eligible operators (42%). 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/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..4ed257661d 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,548 @@ TakeOrderedAndProject +- HashAggregate - +- CometNativeColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Expand - +- Union - :- Project - : +- 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 - : : : : +- 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 - : : +- 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 - : +- 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 + +- Exchange + +- HashAggregate + +- Expand + +- Union + :- Project + : +- Filter + : : +- Subquery + : : +- HashAggregate + : : +- Exchange + : : +- 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 + : +- Exchange + : +- 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 + : : +- ReusedSubquery + : +- HashAggregate + : +- Exchange + : +- 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 + : +- ReusedSubquery + +- HashAggregate + +- Exchange + +- 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 -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 159 out of 458 eligible operators (34%). Final plan contains 88 transitions 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..8d34a30261 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 @@ -3,402 +3,399 @@ TakeOrderedAndProject :- 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 + : : +- Exchange + : : +- 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 + : +- Exchange + : +- 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 + +- Exchange + +- 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 -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 125 out of 333 eligible operators (37%). Final plan contains 66 transitions 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..822dcf6212 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,35 @@ 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 + +- Exchange + +- 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 -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 11 out of 28 eligible operators (39%). 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/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..81047a7757 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,69 @@ 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 + +- Exchange + +- 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 -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 21 out of 57 eligible operators (36%). 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/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..8b93be4c00 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,57 @@ 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 + +- Exchange + +- 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 -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 20 out of 47 eligible operators (42%). 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-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..74b85d1536 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 @@ -5,31 +5,30 @@ TakeOrderedAndProject +- 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 + +- Exchange + +- 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 -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 11 out of 27 eligible operators (40%). 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/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..d0c6d62f37 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,34 @@ 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 + +- Exchange + +- 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 -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 9 out of 27 eligible operators (33%). 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/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..036d0bad77 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,36 @@ 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 + +- Exchange + +- 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 -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 11 out of 29 eligible operators (37%). 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/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..5aa139bc02 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 @@ -23,65 +23,62 @@ CometNativeColumnarToRow : : : +- 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 + : : : +- Exchange + : : : +- 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 : : +- CometSort : : +- CometProject : : +- CometFilter : : : +- Subquery : : : +- HashAggregate - : : : +- CometNativeColumnarToRow - : : : +- CometColumnarExchange + : : : +- Exchange + : : : +- HashAggregate : : : +- 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 + : : : +- Exchange + : : : +- 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 : : +- CometHashAggregate : : +- CometExchange : : +- CometHashAggregate @@ -112,31 +109,30 @@ CometNativeColumnarToRow : : +- 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 + : : +- Exchange + : : +- 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 : +- CometSort : +- CometProject : +- CometFilter @@ -157,4 +153,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 79 out of 138 eligible operators (57%). Final plan contains 16 transitions 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..68e2df37fc 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 @@ -26,65 +26,62 @@ CometNativeColumnarToRow : : : : +- 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 + : : : : +- Exchange + : : : : +- 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 : : : +- CometSort : : : +- CometProject : : : +- CometFilter : : : : +- Subquery : : : : +- HashAggregate - : : : : +- CometNativeColumnarToRow - : : : : +- CometColumnarExchange + : : : : +- Exchange + : : : : +- HashAggregate : : : : +- 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 + : : : : +- Exchange + : : : : +- 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 : : : +- CometHashAggregate : : : +- CometExchange : : : +- CometHashAggregate @@ -142,31 +139,30 @@ CometNativeColumnarToRow : : : +- 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 + : : : +- Exchange + : : : +- 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 : : +- CometSort : : +- CometProject : : +- CometFilter @@ -209,4 +205,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 127 out of 190 eligible operators (66%). Final plan contains 16 transitions 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/q24a.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q24a.native_datafusion/extended.txt index 03e6809902..8249b86a1e 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q24a.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q24a.native_datafusion/extended.txt @@ -1,96 +1,92 @@ Filter : +- Subquery : +- HashAggregate -: +- CometNativeColumnarToRow -: +- CometColumnarExchange +: +- Exchange +: +- HashAggregate : +- HashAggregate -: +- HashAggregate -: +- CometNativeColumnarToRow -: +- CometColumnarExchange -: +- HashAggregate -: +- Project -: +- BroadcastHashJoin [COMET: Comet is not compatible with Spark for case conversion in locale-specific cases. Set spark.comet.caseConversion.enabled=true to enable it anyway.] -: :- CometNativeColumnarToRow -: : +- CometProject -: : +- CometBroadcastHashJoin -: : :- CometProject -: : : +- CometBroadcastHashJoin -: : : :- CometProject -: : : : +- CometBroadcastHashJoin -: : : : :- CometProject -: : : : : +- CometSortMergeJoin -: : : : : :- CometSort -: : : : : : +- CometExchange -: : : : : : +- CometProject -: : : : : : +- CometFilter -: : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales -: : : : : +- CometSort -: : : : : +- CometExchange -: : : : : +- CometProject -: : : : : +- CometFilter -: : : : : +- CometNativeScan parquet spark_catalog.default.store_returns -: : : : +- CometBroadcastExchange -: : : : +- CometProject -: : : : +- CometFilter -: : : : +- CometNativeScan parquet spark_catalog.default.store -: : : +- CometBroadcastExchange -: : : +- CometProject -: : : +- CometFilter -: : : +- CometNativeScan parquet spark_catalog.default.item -: : +- CometBroadcastExchange -: : +- CometProject -: : +- CometFilter -: : +- CometNativeScan parquet spark_catalog.default.customer -: +- BroadcastExchange -: +- CometNativeColumnarToRow -: +- CometProject -: +- CometFilter -: +- CometNativeScan parquet spark_catalog.default.customer_address +: +- Exchange +: +- HashAggregate +: +- Project +: +- BroadcastHashJoin [COMET: Comet is not compatible with Spark for case conversion in locale-specific cases. Set spark.comet.caseConversion.enabled=true to enable it anyway.] +: :- CometNativeColumnarToRow +: : +- CometProject +: : +- CometBroadcastHashJoin +: : :- CometProject +: : : +- CometBroadcastHashJoin +: : : :- CometProject +: : : : +- CometBroadcastHashJoin +: : : : :- CometProject +: : : : : +- CometSortMergeJoin +: : : : : :- CometSort +: : : : : : +- CometExchange +: : : : : : +- CometProject +: : : : : : +- CometFilter +: : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales +: : : : : +- CometSort +: : : : : +- CometExchange +: : : : : +- CometProject +: : : : : +- CometFilter +: : : : : +- CometNativeScan parquet spark_catalog.default.store_returns +: : : : +- CometBroadcastExchange +: : : : +- CometProject +: : : : +- CometFilter +: : : : +- CometNativeScan parquet spark_catalog.default.store +: : : +- CometBroadcastExchange +: : : +- CometProject +: : : +- CometFilter +: : : +- CometNativeScan parquet spark_catalog.default.item +: : +- CometBroadcastExchange +: : +- CometProject +: : +- CometFilter +: : +- CometNativeScan parquet spark_catalog.default.customer +: +- BroadcastExchange +: +- CometNativeColumnarToRow +: +- CometProject +: +- CometFilter +: +- CometNativeScan parquet spark_catalog.default.customer_address +- HashAggregate - +- CometNativeColumnarToRow - +- CometColumnarExchange + +- Exchange + +- HashAggregate +- HashAggregate - +- HashAggregate - +- CometNativeColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Project - +- BroadcastHashJoin [COMET: Comet is not compatible with Spark for case conversion in locale-specific cases. Set spark.comet.caseConversion.enabled=true to enable it anyway.] - :- CometNativeColumnarToRow - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometSortMergeJoin - : : : : :- CometSort - : : : : : +- CometExchange - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : : +- CometSort - : : : : +- CometExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.store_returns - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.store - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.item - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.customer - +- BroadcastExchange - +- CometNativeColumnarToRow - +- CometProject - +- CometFilter - +- CometNativeScan parquet spark_catalog.default.customer_address + +- Exchange + +- HashAggregate + +- Project + +- BroadcastHashJoin [COMET: Comet is not compatible with Spark for case conversion in locale-specific cases. Set spark.comet.caseConversion.enabled=true to enable it anyway.] + :- CometNativeColumnarToRow + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometSortMergeJoin + : : : : :- CometSort + : : : : : +- CometExchange + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : +- CometSort + : : : : +- CometExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.store_returns + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.store + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.item + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.customer + +- BroadcastExchange + +- CometNativeColumnarToRow + +- CometProject + +- CometFilter + +- CometNativeScan parquet spark_catalog.default.customer_address -Comet accelerated 70 out of 86 eligible operators (81%). Final plan contains 8 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 66 out of 86 eligible operators (76%). 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/q24a.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q24a.native_iceberg_compat/extended.txt index 36748b40ea..fa983a36da 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q24a.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q24a.native_iceberg_compat/extended.txt @@ -1,96 +1,92 @@ Filter : +- Subquery : +- HashAggregate -: +- CometNativeColumnarToRow -: +- CometColumnarExchange +: +- Exchange +: +- HashAggregate : +- HashAggregate -: +- HashAggregate -: +- CometNativeColumnarToRow -: +- CometColumnarExchange -: +- HashAggregate -: +- Project -: +- BroadcastHashJoin [COMET: Comet is not compatible with Spark for case conversion in locale-specific cases. Set spark.comet.caseConversion.enabled=true to enable it anyway.] -: :- CometNativeColumnarToRow -: : +- CometProject -: : +- CometBroadcastHashJoin -: : :- CometProject -: : : +- CometBroadcastHashJoin -: : : :- CometProject -: : : : +- CometBroadcastHashJoin -: : : : :- CometProject -: : : : : +- CometSortMergeJoin -: : : : : :- CometSort -: : : : : : +- CometExchange -: : : : : : +- CometProject -: : : : : : +- CometFilter -: : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -: : : : : +- CometSort -: : : : : +- CometExchange -: : : : : +- CometProject -: : : : : +- CometFilter -: : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns -: : : : +- CometBroadcastExchange -: : : : +- CometProject -: : : : +- CometFilter -: : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store -: : : +- CometBroadcastExchange -: : : +- CometProject -: : : +- CometFilter -: : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item -: : +- CometBroadcastExchange -: : +- CometProject -: : +- CometFilter -: : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer -: +- BroadcastExchange -: +- CometNativeColumnarToRow -: +- CometProject -: +- CometFilter -: +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address +: +- Exchange +: +- HashAggregate +: +- Project +: +- BroadcastHashJoin [COMET: Comet is not compatible with Spark for case conversion in locale-specific cases. Set spark.comet.caseConversion.enabled=true to enable it anyway.] +: :- CometNativeColumnarToRow +: : +- CometProject +: : +- CometBroadcastHashJoin +: : :- CometProject +: : : +- CometBroadcastHashJoin +: : : :- CometProject +: : : : +- CometBroadcastHashJoin +: : : : :- CometProject +: : : : : +- CometSortMergeJoin +: : : : : :- CometSort +: : : : : : +- CometExchange +: : : : : : +- CometProject +: : : : : : +- CometFilter +: : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales +: : : : : +- CometSort +: : : : : +- CometExchange +: : : : : +- CometProject +: : : : : +- CometFilter +: : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns +: : : : +- CometBroadcastExchange +: : : : +- CometProject +: : : : +- CometFilter +: : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store +: : : +- CometBroadcastExchange +: : : +- CometProject +: : : +- CometFilter +: : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item +: : +- CometBroadcastExchange +: : +- CometProject +: : +- CometFilter +: : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer +: +- BroadcastExchange +: +- CometNativeColumnarToRow +: +- CometProject +: +- CometFilter +: +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address +- HashAggregate - +- CometNativeColumnarToRow - +- CometColumnarExchange + +- Exchange + +- HashAggregate +- HashAggregate - +- HashAggregate - +- CometNativeColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Project - +- BroadcastHashJoin [COMET: Comet is not compatible with Spark for case conversion in locale-specific cases. Set spark.comet.caseConversion.enabled=true to enable it anyway.] - :- CometNativeColumnarToRow - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometSortMergeJoin - : : : : :- CometSort - : : : : : +- CometExchange - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : +- CometSort - : : : : +- CometExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer - +- BroadcastExchange - +- CometNativeColumnarToRow - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address + +- Exchange + +- HashAggregate + +- Project + +- BroadcastHashJoin [COMET: Comet is not compatible with Spark for case conversion in locale-specific cases. Set spark.comet.caseConversion.enabled=true to enable it anyway.] + :- CometNativeColumnarToRow + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometSortMergeJoin + : : : : :- CometSort + : : : : : +- CometExchange + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales + : : : : +- CometSort + : : : : +- CometExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer + +- BroadcastExchange + +- CometNativeColumnarToRow + +- CometProject + +- CometFilter + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address -Comet accelerated 70 out of 86 eligible operators (81%). Final plan contains 8 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 66 out of 86 eligible operators (76%). 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/q24b.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q24b.native_datafusion/extended.txt index 03e6809902..8249b86a1e 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q24b.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q24b.native_datafusion/extended.txt @@ -1,96 +1,92 @@ Filter : +- Subquery : +- HashAggregate -: +- CometNativeColumnarToRow -: +- CometColumnarExchange +: +- Exchange +: +- HashAggregate : +- HashAggregate -: +- HashAggregate -: +- CometNativeColumnarToRow -: +- CometColumnarExchange -: +- HashAggregate -: +- Project -: +- BroadcastHashJoin [COMET: Comet is not compatible with Spark for case conversion in locale-specific cases. Set spark.comet.caseConversion.enabled=true to enable it anyway.] -: :- CometNativeColumnarToRow -: : +- CometProject -: : +- CometBroadcastHashJoin -: : :- CometProject -: : : +- CometBroadcastHashJoin -: : : :- CometProject -: : : : +- CometBroadcastHashJoin -: : : : :- CometProject -: : : : : +- CometSortMergeJoin -: : : : : :- CometSort -: : : : : : +- CometExchange -: : : : : : +- CometProject -: : : : : : +- CometFilter -: : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales -: : : : : +- CometSort -: : : : : +- CometExchange -: : : : : +- CometProject -: : : : : +- CometFilter -: : : : : +- CometNativeScan parquet spark_catalog.default.store_returns -: : : : +- CometBroadcastExchange -: : : : +- CometProject -: : : : +- CometFilter -: : : : +- CometNativeScan parquet spark_catalog.default.store -: : : +- CometBroadcastExchange -: : : +- CometProject -: : : +- CometFilter -: : : +- CometNativeScan parquet spark_catalog.default.item -: : +- CometBroadcastExchange -: : +- CometProject -: : +- CometFilter -: : +- CometNativeScan parquet spark_catalog.default.customer -: +- BroadcastExchange -: +- CometNativeColumnarToRow -: +- CometProject -: +- CometFilter -: +- CometNativeScan parquet spark_catalog.default.customer_address +: +- Exchange +: +- HashAggregate +: +- Project +: +- BroadcastHashJoin [COMET: Comet is not compatible with Spark for case conversion in locale-specific cases. Set spark.comet.caseConversion.enabled=true to enable it anyway.] +: :- CometNativeColumnarToRow +: : +- CometProject +: : +- CometBroadcastHashJoin +: : :- CometProject +: : : +- CometBroadcastHashJoin +: : : :- CometProject +: : : : +- CometBroadcastHashJoin +: : : : :- CometProject +: : : : : +- CometSortMergeJoin +: : : : : :- CometSort +: : : : : : +- CometExchange +: : : : : : +- CometProject +: : : : : : +- CometFilter +: : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales +: : : : : +- CometSort +: : : : : +- CometExchange +: : : : : +- CometProject +: : : : : +- CometFilter +: : : : : +- CometNativeScan parquet spark_catalog.default.store_returns +: : : : +- CometBroadcastExchange +: : : : +- CometProject +: : : : +- CometFilter +: : : : +- CometNativeScan parquet spark_catalog.default.store +: : : +- CometBroadcastExchange +: : : +- CometProject +: : : +- CometFilter +: : : +- CometNativeScan parquet spark_catalog.default.item +: : +- CometBroadcastExchange +: : +- CometProject +: : +- CometFilter +: : +- CometNativeScan parquet spark_catalog.default.customer +: +- BroadcastExchange +: +- CometNativeColumnarToRow +: +- CometProject +: +- CometFilter +: +- CometNativeScan parquet spark_catalog.default.customer_address +- HashAggregate - +- CometNativeColumnarToRow - +- CometColumnarExchange + +- Exchange + +- HashAggregate +- HashAggregate - +- HashAggregate - +- CometNativeColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Project - +- BroadcastHashJoin [COMET: Comet is not compatible with Spark for case conversion in locale-specific cases. Set spark.comet.caseConversion.enabled=true to enable it anyway.] - :- CometNativeColumnarToRow - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometSortMergeJoin - : : : : :- CometSort - : : : : : +- CometExchange - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : : +- CometSort - : : : : +- CometExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.store_returns - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.store - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.item - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.customer - +- BroadcastExchange - +- CometNativeColumnarToRow - +- CometProject - +- CometFilter - +- CometNativeScan parquet spark_catalog.default.customer_address + +- Exchange + +- HashAggregate + +- Project + +- BroadcastHashJoin [COMET: Comet is not compatible with Spark for case conversion in locale-specific cases. Set spark.comet.caseConversion.enabled=true to enable it anyway.] + :- CometNativeColumnarToRow + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometSortMergeJoin + : : : : :- CometSort + : : : : : +- CometExchange + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : +- CometSort + : : : : +- CometExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.store_returns + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.store + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.item + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.customer + +- BroadcastExchange + +- CometNativeColumnarToRow + +- CometProject + +- CometFilter + +- CometNativeScan parquet spark_catalog.default.customer_address -Comet accelerated 70 out of 86 eligible operators (81%). Final plan contains 8 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 66 out of 86 eligible operators (76%). 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/q24b.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q24b.native_iceberg_compat/extended.txt index 36748b40ea..fa983a36da 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q24b.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q24b.native_iceberg_compat/extended.txt @@ -1,96 +1,92 @@ Filter : +- Subquery : +- HashAggregate -: +- CometNativeColumnarToRow -: +- CometColumnarExchange +: +- Exchange +: +- HashAggregate : +- HashAggregate -: +- HashAggregate -: +- CometNativeColumnarToRow -: +- CometColumnarExchange -: +- HashAggregate -: +- Project -: +- BroadcastHashJoin [COMET: Comet is not compatible with Spark for case conversion in locale-specific cases. Set spark.comet.caseConversion.enabled=true to enable it anyway.] -: :- CometNativeColumnarToRow -: : +- CometProject -: : +- CometBroadcastHashJoin -: : :- CometProject -: : : +- CometBroadcastHashJoin -: : : :- CometProject -: : : : +- CometBroadcastHashJoin -: : : : :- CometProject -: : : : : +- CometSortMergeJoin -: : : : : :- CometSort -: : : : : : +- CometExchange -: : : : : : +- CometProject -: : : : : : +- CometFilter -: : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -: : : : : +- CometSort -: : : : : +- CometExchange -: : : : : +- CometProject -: : : : : +- CometFilter -: : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns -: : : : +- CometBroadcastExchange -: : : : +- CometProject -: : : : +- CometFilter -: : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store -: : : +- CometBroadcastExchange -: : : +- CometProject -: : : +- CometFilter -: : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item -: : +- CometBroadcastExchange -: : +- CometProject -: : +- CometFilter -: : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer -: +- BroadcastExchange -: +- CometNativeColumnarToRow -: +- CometProject -: +- CometFilter -: +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address +: +- Exchange +: +- HashAggregate +: +- Project +: +- BroadcastHashJoin [COMET: Comet is not compatible with Spark for case conversion in locale-specific cases. Set spark.comet.caseConversion.enabled=true to enable it anyway.] +: :- CometNativeColumnarToRow +: : +- CometProject +: : +- CometBroadcastHashJoin +: : :- CometProject +: : : +- CometBroadcastHashJoin +: : : :- CometProject +: : : : +- CometBroadcastHashJoin +: : : : :- CometProject +: : : : : +- CometSortMergeJoin +: : : : : :- CometSort +: : : : : : +- CometExchange +: : : : : : +- CometProject +: : : : : : +- CometFilter +: : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales +: : : : : +- CometSort +: : : : : +- CometExchange +: : : : : +- CometProject +: : : : : +- CometFilter +: : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns +: : : : +- CometBroadcastExchange +: : : : +- CometProject +: : : : +- CometFilter +: : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store +: : : +- CometBroadcastExchange +: : : +- CometProject +: : : +- CometFilter +: : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item +: : +- CometBroadcastExchange +: : +- CometProject +: : +- CometFilter +: : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer +: +- BroadcastExchange +: +- CometNativeColumnarToRow +: +- CometProject +: +- CometFilter +: +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address +- HashAggregate - +- CometNativeColumnarToRow - +- CometColumnarExchange + +- Exchange + +- HashAggregate +- HashAggregate - +- HashAggregate - +- CometNativeColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Project - +- BroadcastHashJoin [COMET: Comet is not compatible with Spark for case conversion in locale-specific cases. Set spark.comet.caseConversion.enabled=true to enable it anyway.] - :- CometNativeColumnarToRow - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometSortMergeJoin - : : : : :- CometSort - : : : : : +- CometExchange - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : +- CometSort - : : : : +- CometExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer - +- BroadcastExchange - +- CometNativeColumnarToRow - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address + +- Exchange + +- HashAggregate + +- Project + +- BroadcastHashJoin [COMET: Comet is not compatible with Spark for case conversion in locale-specific cases. Set spark.comet.caseConversion.enabled=true to enable it anyway.] + :- CometNativeColumnarToRow + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometSortMergeJoin + : : : : :- CometSort + : : : : : +- CometExchange + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales + : : : : +- CometSort + : : : : +- CometExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer + +- BroadcastExchange + +- CometNativeColumnarToRow + +- CometProject + +- CometFilter + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address -Comet accelerated 70 out of 86 eligible operators (81%). Final plan contains 8 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 66 out of 86 eligible operators (76%). 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/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..81047a7757 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,69 @@ 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 + +- Exchange + +- 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 -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 21 out of 57 eligible operators (36%). 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/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..9a298d216c 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,43 @@ 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 + +- Exchange + +- 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 -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 15 out of 35 eligible operators (42%). 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/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..87d8460ec7 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,44 @@ 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 + +- Exchange + +- 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 -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 15 out of 36 eligible operators (41%). 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/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..98d6e2d24e 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,74 @@ 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 + +- Exchange + +- 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 -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 24 out of 61 eligible operators (39%). 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/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..659f71490a 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 @@ -7,60 +7,57 @@ TakeOrderedAndProject : : +- 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 + : : : +- Exchange + : : : +- 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 + : : +- Exchange + : : +- HashAggregate : : +- 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 + : : +- Exchange + : : +- 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 @@ -72,4 +69,4 @@ TakeOrderedAndProject +- 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 21 out of 61 eligible operators (34%). 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/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..9b960f1ab1 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 @@ -10,141 +10,135 @@ CometNativeColumnarToRow : : : +- 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 + : : : : : +- Exchange + : : : : : +- 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 + : : : : +- Exchange + : : : : +- 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 + : : : +- Exchange + : : : +- 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 + : : +- Exchange + : : +- 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 + : +- Exchange + : +- 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 + +- Exchange + +- 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 -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 32 out of 120 eligible operators (26%). 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-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..69d5009252 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,46 @@ 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 ++- Exchange + +- 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 + : +- Exchange + : +- 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 -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 12 out of 38 eligible operators (31%). 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/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..b237976296 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,108 @@ 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 + +- Exchange + +- HashAggregate + +- Union + :- HashAggregate + : +- Exchange + : +- 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 + : +- Exchange + : +- 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 + +- Exchange + +- 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 -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 42 out of 93 eligible operators (45%). 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-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..575c853e76 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 @@ -5,43 +5,42 @@ CometNativeColumnarToRow +- 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 + : +- Exchange + : +- 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 -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 17 out of 37 eligible operators (45%). 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/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..c8a9740a02 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 @@ -1,67 +1,66 @@ TakeOrderedAndProject +- HashAggregate - +- CometNativeColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- 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 - : : : : +- 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 - : : : +- 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 - : : +- 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 + +- Exchange + +- HashAggregate + +- Project + +- BroadcastHashJoin + :- Project + : +- BroadcastHashJoin + : :- 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 + : : : : +- 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 + : : : +- 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 + : : +- 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 -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 20 out of 54 eligible operators (37%). 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/q35.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q35.native_iceberg_compat/extended.txt index c5125b1861..72960fc267 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q35.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q35.native_iceberg_compat/extended.txt @@ -1,63 +1,62 @@ TakeOrderedAndProject +- HashAggregate - +- CometNativeColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- Project - : : +- Filter - : : +- BroadcastHashJoin - : : :- BroadcastHashJoin [COMET: Unsupported join type ExistenceJoin(exists#1)] - : : : :- CometNativeColumnarToRow - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : +- ReusedSubquery - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : +- ReusedSubquery - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- CometNativeColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address - +- BroadcastExchange - +- CometNativeColumnarToRow - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics + +- Exchange + +- HashAggregate + +- Project + +- BroadcastHashJoin + :- Project + : +- BroadcastHashJoin + : :- Project + : : +- Filter + : : +- BroadcastHashJoin + : : :- BroadcastHashJoin [COMET: Unsupported join type ExistenceJoin(exists#1)] + : : : :- CometNativeColumnarToRow + : : : : +- CometBroadcastHashJoin + : : : : :- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometNativeColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : +- BroadcastExchange + : : : +- CometNativeColumnarToRow + : : : +- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales + : : : : +- ReusedSubquery + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- BroadcastExchange + : : +- CometNativeColumnarToRow + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales + : : : +- ReusedSubquery + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- BroadcastExchange + : +- CometNativeColumnarToRow + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address + +- BroadcastExchange + +- CometNativeColumnarToRow + +- CometProject + +- CometFilter + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics -Comet accelerated 35 out of 54 eligible operators (64%). Final plan contains 7 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 34 out of 54 eligible operators (62%). 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 aa40ed68cc..7b401d7eb8 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 @@ -5,39 +5,38 @@ TakeOrderedAndProject +- 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 + +- Exchange + +- 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 -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 14 out of 34 eligible operators (41%). 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/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..c38a41ba07 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 @@ -5,71 +5,69 @@ CometNativeColumnarToRow :- 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 + : +- Exchange + : +- 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 + +- Exchange + +- 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 -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 22 out of 60 eligible operators (36%). 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/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..c38a41ba07 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 @@ -5,71 +5,69 @@ CometNativeColumnarToRow :- 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 + : +- Exchange + : +- 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 + +- Exchange + +- 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 -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 22 out of 60 eligible operators (36%). 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/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..d28d212f33 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 @@ -10,145 +10,139 @@ TakeOrderedAndProject : : : :- 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 + : : : : : +- Exchange + : : : : : +- 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 + : : : : +- Exchange + : : : : +- 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 + : : : +- Exchange + : : : +- 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 + : : +- Exchange + : : +- 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 + : +- Exchange + : +- 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 + +- Exchange + +- 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 -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 34 out of 126 eligible operators (26%). Final plan contains 20 transitions 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..4f2175cf7b 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 @@ -1,51 +1,50 @@ TakeOrderedAndProject +- HashAggregate - +- CometNativeColumnarToRow - +- CometColumnarExchange - +- 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 - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.item - +- BroadcastExchange - +- CometNativeColumnarToRow - +- CometProject - +- CometFilter - +- CometNativeScan parquet spark_catalog.default.item + +- Exchange + +- 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 + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.item + +- BroadcastExchange + +- CometNativeColumnarToRow + +- CometProject + +- 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 17 out of 41 eligible operators (41%). 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/q45.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q45.native_iceberg_compat/extended.txt index afdf8d6633..f8c94605ab 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q45.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q45.native_iceberg_compat/extended.txt @@ -1,47 +1,46 @@ TakeOrderedAndProject +- HashAggregate - +- CometNativeColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Project - +- Filter - +- BroadcastHashJoin [COMET: Unsupported join type ExistenceJoin(exists#1)] - :- CometNativeColumnarToRow - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - +- BroadcastExchange - +- CometNativeColumnarToRow - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + +- Exchange + +- HashAggregate + +- Project + +- Filter + +- BroadcastHashJoin [COMET: Unsupported join type ExistenceJoin(exists#1)] + :- CometNativeColumnarToRow + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometNativeColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + +- BroadcastExchange + +- CometNativeColumnarToRow + +- CometProject + +- CometFilter + +- CometScan [native_iceberg_compat] 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 31 out of 41 eligible operators (75%). 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 cebcd85144..7aaa58fcbf 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 @@ -4,45 +4,44 @@ TakeOrderedAndProject :- 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 + : : +- Exchange + : : +- 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 @@ -53,4 +52,4 @@ TakeOrderedAndProject +- 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 19 out of 45 eligible operators (42%). 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-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..e347d0f433 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 @@ -12,36 +12,35 @@ TakeOrderedAndProject : : +- 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 + : : +- Exchange + : : +- 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 : +- 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).] @@ -49,36 +48,35 @@ TakeOrderedAndProject : +- 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 + : +- Exchange + : +- 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 +- 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).] @@ -86,35 +84,34 @@ TakeOrderedAndProject +- 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 + +- Exchange + +- 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 -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 33 out of 97 eligible operators (34%). 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-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..411a47f8ed 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,41 @@ 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 ++- Exchange + +- 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 -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 14 out of 33 eligible operators (42%). 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/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..d19eb04caf 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 @@ -13,33 +13,32 @@ 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 + : +- Exchange + : +- 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 :- Project : +- Filter : +- Window @@ -49,28 +48,27 @@ 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 + : +- Exchange + : +- 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 +- Project +- Filter +- Window @@ -80,27 +78,26 @@ 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 + +- Exchange + +- 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 -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 30 out of 87 eligible operators (34%). 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-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..c9204f8e91 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,102 @@ 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 + +- Exchange + +- HashAggregate + +- Expand + +- Union + :- HashAggregate + : +- Exchange + : +- 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 + : +- Exchange + : +- 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 + +- Exchange + +- 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 -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 24 out of 86 eligible operators (27%). 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-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..d0b7b01cee 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,41 @@ 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 + +- Exchange + +- 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 -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 13 out of 33 eligible operators (39%). 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/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..ee122e3dcd 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 @@ -14,25 +14,24 @@ TakeOrderedAndProject : +- 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 + : +- Exchange + : +- 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 +- CometSort +- CometColumnarExchange +- Project @@ -41,19 +40,18 @@ TakeOrderedAndProject +- 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 + +- Exchange + +- 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 -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 21 out of 47 eligible operators (44%). 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-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..9b3651b3fb 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 @@ -6,37 +6,36 @@ TakeOrderedAndProject +- 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 + +- Exchange + +- 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 -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 13 out of 33 eligible operators (39%). 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/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..2d4a7b9185 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,114 @@ TakeOrderedAndProject +- HashAggregate - +- CometNativeColumnarToRow - +- CometColumnarExchange + +- Exchange + +- HashAggregate +- 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 + +- Exchange + +- 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 -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 49 out of 96 eligible operators (51%). Final plan contains 16 transitions 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..4babc2254d 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,111 @@ 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 + +- Exchange + +- HashAggregate + +- Union + :- HashAggregate + : +- Exchange + : +- 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 + : +- Exchange + : +- 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 + +- Exchange + +- 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 -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 45 out of 96 eligible operators (46%). 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-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..50dc55f674 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 @@ -12,36 +12,35 @@ TakeOrderedAndProject : : +- 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 + : : +- Exchange + : : +- 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 : +- 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).] @@ -49,36 +48,35 @@ TakeOrderedAndProject : +- 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 + : +- Exchange + : +- 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 +- 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).] @@ -86,35 +84,34 @@ TakeOrderedAndProject +- 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 + +- Exchange + +- 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 -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 33 out of 97 eligible operators (34%). 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-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..dad556083c 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 @@ -5,119 +5,116 @@ TakeOrderedAndProject : +- 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 + : : +- Exchange + : : +- 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 + : +- Exchange + : +- 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 + +- Exchange + +- 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 -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 53 out of 104 eligible operators (50%). 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-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..18cc2a5c96 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,67 @@ 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 + +- Exchange + +- 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 -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 38 out of 58 eligible operators (65%). 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/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..4babc2254d 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,111 @@ 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 + +- Exchange + +- HashAggregate + +- Union + :- HashAggregate + : +- Exchange + : +- 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 + : +- Exchange + : +- 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 + +- Exchange + +- 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 -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 45 out of 96 eligible operators (46%). 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-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..da7b7fc2f0 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,99 @@ 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 + : +- Exchange + : +- 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 +- 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 + +- Exchange + +- 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 -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 34 out of 83 eligible operators (40%). 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-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..9b3651b3fb 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 @@ -6,37 +6,36 @@ TakeOrderedAndProject +- 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 + +- Exchange + +- 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 -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 13 out of 33 eligible operators (39%). 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/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..f059866684 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 @@ -11,25 +11,24 @@ TakeOrderedAndProject : : +- 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 + : : +- Exchange + : : +- 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 @@ -38,23 +37,21 @@ TakeOrderedAndProject +- BroadcastExchange +- Filter +- HashAggregate - +- CometNativeColumnarToRow - +- CometColumnarExchange + +- Exchange + +- HashAggregate +- 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 + +- Exchange + +- 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 -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 14 out of 48 eligible operators (29%). 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/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..b586928ede 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,79 @@ 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 + +- Exchange + +- HashAggregate + +- Union + :- HashAggregate + : +- Exchange + : +- 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 + +- Exchange + +- 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 -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 24 out of 66 eligible operators (36%). 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/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..611eb3c40f 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 @@ -8,39 +8,38 @@ TakeOrderedAndProject +- 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 + +- Exchange + +- 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 -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 14 out of 37 eligible operators (37%). 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/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..7aaa58fcbf 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 @@ -4,45 +4,44 @@ TakeOrderedAndProject :- 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 + : : +- Exchange + : : +- 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 @@ -53,4 +52,4 @@ TakeOrderedAndProject +- 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 19 out of 45 eligible operators (42%). 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-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..c8bb5a4bde 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 @@ -1,66 +1,65 @@ TakeOrderedAndProject +- HashAggregate - +- CometNativeColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Project - +- BroadcastHashJoin - :- Project - : +- 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 - : : : : +- 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 - : : : +- 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 - : : +- 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 + +- Exchange + +- HashAggregate + +- Project + +- BroadcastHashJoin + :- Project + : +- 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 + : : : : +- 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 + : : : +- 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 + : : +- 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 -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 20 out of 53 eligible operators (37%). 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/q69.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q69.native_iceberg_compat/extended.txt index bf624b5ce3..726fb6b283 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q69.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q69.native_iceberg_compat/extended.txt @@ -1,62 +1,61 @@ TakeOrderedAndProject +- HashAggregate - +- CometNativeColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- BroadcastHashJoin [COMET: BuildRight with LeftAnti is not supported] - : : : :- CometNativeColumnarToRow - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : +- ReusedSubquery - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : +- ReusedSubquery - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- CometNativeColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address - +- BroadcastExchange - +- CometNativeColumnarToRow - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics + +- Exchange + +- HashAggregate + +- Project + +- BroadcastHashJoin + :- Project + : +- BroadcastHashJoin + : :- Project + : : +- BroadcastHashJoin + : : :- BroadcastHashJoin [COMET: BuildRight with LeftAnti is not supported] + : : : :- CometNativeColumnarToRow + : : : : +- CometBroadcastHashJoin + : : : : :- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometNativeColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : +- BroadcastExchange + : : : +- CometNativeColumnarToRow + : : : +- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales + : : : : +- ReusedSubquery + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- BroadcastExchange + : : +- CometNativeColumnarToRow + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales + : : : +- ReusedSubquery + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- BroadcastExchange + : +- CometNativeColumnarToRow + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address + +- BroadcastExchange + +- CometNativeColumnarToRow + +- CometProject + +- CometFilter + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics -Comet accelerated 35 out of 53 eligible operators (66%). Final plan contains 7 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 34 out of 53 eligible operators (64%). 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 b692dfa9be..60c0417071 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,43 @@ 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 + +- Exchange + +- 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 -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 15 out of 35 eligible operators (42%). 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 f222353c17..1e004dfc43 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 @@ -5,61 +5,59 @@ TakeOrderedAndProject +- CometSort +- CometColumnarExchange +- HashAggregate - +- CometNativeColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- 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 - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.date_dim - +- BroadcastExchange - +- Project - +- BroadcastHashJoin - :- CometNativeColumnarToRow - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.store - +- BroadcastExchange - +- 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 + +- Exchange + +- HashAggregate + +- 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 + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.date_dim + +- BroadcastExchange + +- Project + +- BroadcastHashJoin + :- CometNativeColumnarToRow + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.store + +- BroadcastExchange + +- Project + +- Filter + +- Window + +- WindowGroupLimit + +- Sort + +- HashAggregate + +- Exchange + +- 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 -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 16 out of 53 eligible operators (30%). 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-spark3_5/q70.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q70.native_iceberg_compat/extended.txt index 95077efbc5..1ce50f8385 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q70.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q70.native_iceberg_compat/extended.txt @@ -5,57 +5,56 @@ TakeOrderedAndProject +- CometSort +- CometColumnarExchange +- HashAggregate - +- CometNativeColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Expand - +- Project - +- BroadcastHashJoin - :- CometNativeColumnarToRow - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - +- BroadcastExchange - +- Project - +- BroadcastHashJoin - :- CometNativeColumnarToRow - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - +- BroadcastExchange - +- Project - +- Filter - +- Window - +- WindowGroupLimit [COMET: WindowGroupLimit is not supported] - +- CometNativeColumnarToRow - +- CometSort - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : +- ReusedSubquery - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + +- Exchange + +- HashAggregate + +- Expand + +- Project + +- BroadcastHashJoin + :- CometNativeColumnarToRow + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales + : : +- SubqueryBroadcast + : : +- BroadcastExchange + : : +- CometNativeColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + +- BroadcastExchange + +- Project + +- BroadcastHashJoin + :- CometNativeColumnarToRow + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store + +- BroadcastExchange + +- Project + +- Filter + +- Window + +- WindowGroupLimit [COMET: WindowGroupLimit is not supported] + +- CometNativeColumnarToRow + +- CometSort + +- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales + : : +- ReusedSubquery + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store + +- CometBroadcastExchange + +- CometProject + +- CometFilter + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Comet accelerated 34 out of 53 eligible operators (64%). Final plan contains 6 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 33 out of 53 eligible operators (62%). 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 23844f41f3..9d5665df4d 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 @@ -2,61 +2,60 @@ 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 + +- Exchange + +- 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 -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 20 out of 49 eligible operators (40%). 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/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..575c853e76 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 @@ -5,43 +5,42 @@ CometNativeColumnarToRow +- 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 + : +- Exchange + : +- 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 -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 17 out of 37 eligible operators (45%). 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/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..00d2b3b0d7 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 @@ -6,100 +6,96 @@ TakeOrderedAndProject : :- 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 + : : : +- Exchange + : : : +- 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 + : : +- Exchange + : : +- 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 + : +- Exchange + : +- 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 + +- Exchange + +- 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 -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 24 out of 85 eligible operators (28%). 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-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..e647d8db46 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 @@ -1,135 +1,128 @@ TakeOrderedAndProject +- HashAggregate - +- CometNativeColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Expand - +- Union - :- 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 - : :- 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 + +- Exchange + +- HashAggregate + +- Expand + +- Union + :- Project + : +- BroadcastHashJoin + : :- HashAggregate + : : +- Exchange + : : +- 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 + : +- Exchange + : +- 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 + : :- BroadcastExchange + : : +- HashAggregate + : : +- Exchange + : : +- 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 + : +- Exchange + : +- 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 + : +- Exchange + : +- 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 + +- Exchange + +- 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 -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 29 out of 109 eligible operators (26%). Final plan contains 17 transitions 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_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q77.native_iceberg_compat/extended.txt index 0cfc0a8417..b2653c7068 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q77.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q77.native_iceberg_compat/extended.txt @@ -1,117 +1,116 @@ TakeOrderedAndProject +- HashAggregate - +- CometNativeColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Expand - +- Union - :- CometNativeColumnarToRow - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometHashAggregate - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - : +- CometBroadcastExchange - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns - : : : +- ReusedSubquery - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - :- Project - : +- BroadcastNestedLoopJoin [COMET: BroadcastNestedLoopJoin is not supported] - : :- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometHashAggregate - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : +- ReusedSubquery - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometNativeColumnarToRow - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns - : : +- ReusedSubquery - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - +- CometNativeColumnarToRow - +- CometProject - +- CometBroadcastHashJoin - :- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : +- ReusedSubquery - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_page - +- CometBroadcastExchange - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns - : : +- ReusedSubquery - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - +- CometBroadcastExchange - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_page + +- Exchange + +- HashAggregate + +- Expand + +- Union + :- CometNativeColumnarToRow + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometHashAggregate + : : +- CometExchange + : : +- CometHashAggregate + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometNativeColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store + : +- CometBroadcastExchange + : +- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns + : : : +- ReusedSubquery + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store + :- Project + : +- BroadcastNestedLoopJoin [COMET: BroadcastNestedLoopJoin is not supported] + : :- BroadcastExchange + : : +- CometNativeColumnarToRow + : : +- CometHashAggregate + : : +- CometExchange + : : +- CometHashAggregate + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales + : : : +- ReusedSubquery + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- CometNativeColumnarToRow + : +- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns + : : +- ReusedSubquery + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + +- CometNativeColumnarToRow + +- CometProject + +- CometBroadcastHashJoin + :- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales + : : : +- ReusedSubquery + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_page + +- CometBroadcastExchange + +- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns + : : +- ReusedSubquery + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + +- CometBroadcastExchange + +- CometFilter + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_page -Comet accelerated 94 out of 109 eligible operators (86%). Final plan contains 6 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 93 out of 109 eligible operators (85%). 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/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..76bf23f16f 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 @@ -2,43 +2,42 @@ 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 + : +- Exchange + : +- 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 -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 15 out of 35 eligible operators (42%). 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/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..838592f73b 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,55 @@ 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 + +- Exchange + +- 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 -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 31 out of 48 eligible operators (64%). 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/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..6133951418 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 @@ -7,60 +7,57 @@ TakeOrderedAndProject : : +- 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 + : : : +- Exchange + : : : +- 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 + : : +- Exchange + : : +- HashAggregate : : +- 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 + : : +- Exchange + : : +- 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 @@ -72,4 +69,4 @@ TakeOrderedAndProject +- 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 21 out of 61 eligible operators (34%). 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/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..66a0c08b10 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 @@ -4,113 +4,110 @@ TakeOrderedAndProject :- 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 + : : +- Exchange + : : +- 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 + : +- Exchange + : +- 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 + +- Exchange + +- 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 -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 57 out of 101 eligible operators (56%). 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/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..efab56234e 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,63 @@ 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 + +- Exchange + +- 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 -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 23 out of 52 eligible operators (44%). 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/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..7313f7e979 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 @@ -5,32 +5,31 @@ TakeOrderedAndProject +- 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 + +- Exchange + +- 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 -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 11 out of 28 eligible operators (39%). 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/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..772b0f4512 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 @@ -1,82 +1,81 @@ HashAggregate -+- CometNativeColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Project - +- BroadcastHashJoin - :- 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 - : +- 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 - +- 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 ++- Exchange + +- HashAggregate + +- Project + +- BroadcastHashJoin + :- 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 + : +- 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 + +- 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 -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 27 out of 66 eligible operators (40%). 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-spark3_5/q87.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q87.native_iceberg_compat/extended.txt index 5fddd74768..6218729c98 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q87.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q87.native_iceberg_compat/extended.txt @@ -1,73 +1,72 @@ HashAggregate -+- CometNativeColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Project - +- BroadcastHashJoin - :- BroadcastHashJoin [COMET: BuildRight with LeftAnti is not supported] - : :- CometNativeColumnarToRow - : : +- CometHashAggregate - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer - : +- BroadcastExchange - : +- CometNativeColumnarToRow - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : +- ReusedSubquery - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer - +- BroadcastExchange - +- CometNativeColumnarToRow - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : +- ReusedSubquery - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer ++- Exchange + +- HashAggregate + +- Project + +- BroadcastHashJoin + :- BroadcastHashJoin [COMET: BuildRight with LeftAnti is not supported] + : :- CometNativeColumnarToRow + : : +- CometHashAggregate + : : +- CometExchange + : : +- CometHashAggregate + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometNativeColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer + : +- BroadcastExchange + : +- CometNativeColumnarToRow + : +- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales + : : : +- ReusedSubquery + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer + +- BroadcastExchange + +- CometNativeColumnarToRow + +- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales + : : +- ReusedSubquery + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + +- CometBroadcastExchange + +- CometProject + +- CometFilter + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer -Comet accelerated 55 out of 66 eligible operators (83%). Final plan contains 5 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 54 out of 66 eligible operators (81%). 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 5c7a26057d..9b3651b3fb 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 @@ -6,37 +6,36 @@ TakeOrderedAndProject +- 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 + +- Exchange + +- 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 -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 13 out of 33 eligible operators (39%). 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/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..27bfd61c53 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 @@ -2,58 +2,57 @@ 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 + +- Exchange + +- 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 -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 22 out of 47 eligible operators (46%). 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/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..c0f40501d8 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,46 @@ 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 ++- Exchange + +- 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 + : +- Exchange + : +- 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 -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 12 out of 38 eligible operators (31%). 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/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..9de7969459 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 @@ -8,31 +8,30 @@ 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 + +- Exchange + +- 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 -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 14 out of 29 eligible operators (48%). 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/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..a3434fe3ea 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 @@ -7,46 +7,43 @@ TakeOrderedAndProject : : +- 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 + : : : +- Exchange + : : : +- 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 + : : +- Exchange + : : +- HashAggregate : : +- 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 + : : +- Exchange + : : +- 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 @@ -58,4 +55,4 @@ TakeOrderedAndProject +- 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 15 out of 49 eligible operators (30%). 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/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..c8a9740a02 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 @@ -1,67 +1,66 @@ TakeOrderedAndProject +- HashAggregate - +- CometNativeColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- 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 - : : : : +- 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 - : : : +- 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 - : : +- 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 + +- Exchange + +- HashAggregate + +- Project + +- BroadcastHashJoin + :- Project + : +- BroadcastHashJoin + : :- 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 + : : : : +- 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 + : : : +- 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 + : : +- 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 -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 20 out of 54 eligible operators (37%). 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/q10.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q10.native_iceberg_compat/extended.txt index c5125b1861..72960fc267 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q10.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q10.native_iceberg_compat/extended.txt @@ -1,63 +1,62 @@ TakeOrderedAndProject +- HashAggregate - +- CometNativeColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- Project - : : +- Filter - : : +- BroadcastHashJoin - : : :- BroadcastHashJoin [COMET: Unsupported join type ExistenceJoin(exists#1)] - : : : :- CometNativeColumnarToRow - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : +- ReusedSubquery - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : +- ReusedSubquery - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- CometNativeColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address - +- BroadcastExchange - +- CometNativeColumnarToRow - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics + +- Exchange + +- HashAggregate + +- Project + +- BroadcastHashJoin + :- Project + : +- BroadcastHashJoin + : :- Project + : : +- Filter + : : +- BroadcastHashJoin + : : :- BroadcastHashJoin [COMET: Unsupported join type ExistenceJoin(exists#1)] + : : : :- CometNativeColumnarToRow + : : : : +- CometBroadcastHashJoin + : : : : :- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometNativeColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : +- BroadcastExchange + : : : +- CometNativeColumnarToRow + : : : +- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales + : : : : +- ReusedSubquery + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- BroadcastExchange + : : +- CometNativeColumnarToRow + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales + : : : +- ReusedSubquery + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- BroadcastExchange + : +- CometNativeColumnarToRow + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address + +- BroadcastExchange + +- CometNativeColumnarToRow + +- CometProject + +- CometFilter + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics -Comet accelerated 35 out of 54 eligible operators (64%). Final plan contains 7 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 34 out of 54 eligible operators (62%). 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 47aa9517cf..a38205d189 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 @@ -7,100 +7,96 @@ TakeOrderedAndProject : : +- 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 + : : : +- Exchange + : : : +- 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 + : : +- Exchange + : : +- 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 + : +- Exchange + : +- 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 + +- Exchange + +- 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 -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 24 out of 86 eligible operators (27%). 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-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..79e4955126 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 @@ -5,31 +5,30 @@ TakeOrderedAndProject +- 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 + +- Exchange + +- 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 -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 11 out of 27 eligible operators (40%). 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/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..886c7fc5fe 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,47 @@ 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 ++- Exchange + +- 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 -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 16 out of 38 eligible operators (42%). 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/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..4ed257661d 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,548 @@ TakeOrderedAndProject +- HashAggregate - +- CometNativeColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Expand - +- Union - :- Project - : +- 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 - : : : : +- 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 - : : +- 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 - : +- 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 + +- Exchange + +- HashAggregate + +- Expand + +- Union + :- Project + : +- Filter + : : +- Subquery + : : +- HashAggregate + : : +- Exchange + : : +- 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 + : +- Exchange + : +- 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 + : : +- ReusedSubquery + : +- HashAggregate + : +- Exchange + : +- 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 + : +- ReusedSubquery + +- HashAggregate + +- Exchange + +- 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 -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 159 out of 458 eligible operators (34%). Final plan contains 88 transitions 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..30eef295ec 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 @@ -3,406 +3,403 @@ TakeOrderedAndProject :- 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 + : : +- Exchange + : : +- 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 + : +- Exchange + : +- 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 + +- Exchange + +- 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 -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 125 out of 337 eligible operators (37%). Final plan contains 66 transitions 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..822dcf6212 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,35 @@ 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 + +- Exchange + +- 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 -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 11 out of 28 eligible operators (39%). 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/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..81047a7757 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,69 @@ 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 + +- Exchange + +- 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 -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 21 out of 57 eligible operators (36%). 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/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..8b93be4c00 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,57 @@ 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 + +- Exchange + +- 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 -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 20 out of 47 eligible operators (42%). 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/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..74b85d1536 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 @@ -5,31 +5,30 @@ TakeOrderedAndProject +- 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 + +- Exchange + +- 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 -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 11 out of 27 eligible operators (40%). 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/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..d0c6d62f37 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,34 @@ 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 + +- Exchange + +- 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 -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 9 out of 27 eligible operators (33%). 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/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..036d0bad77 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,36 @@ 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 + +- Exchange + +- 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 -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 11 out of 29 eligible operators (37%). 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/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..5aa139bc02 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 @@ -23,65 +23,62 @@ CometNativeColumnarToRow : : : +- 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 + : : : +- Exchange + : : : +- 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 : : +- CometSort : : +- CometProject : : +- CometFilter : : : +- Subquery : : : +- HashAggregate - : : : +- CometNativeColumnarToRow - : : : +- CometColumnarExchange + : : : +- Exchange + : : : +- HashAggregate : : : +- 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 + : : : +- Exchange + : : : +- 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 : : +- CometHashAggregate : : +- CometExchange : : +- CometHashAggregate @@ -112,31 +109,30 @@ CometNativeColumnarToRow : : +- 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 + : : +- Exchange + : : +- 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 : +- CometSort : +- CometProject : +- CometFilter @@ -157,4 +153,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 79 out of 138 eligible operators (57%). Final plan contains 16 transitions 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..68e2df37fc 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 @@ -26,65 +26,62 @@ CometNativeColumnarToRow : : : : +- 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 + : : : : +- Exchange + : : : : +- 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 : : : +- CometSort : : : +- CometProject : : : +- CometFilter : : : : +- Subquery : : : : +- HashAggregate - : : : : +- CometNativeColumnarToRow - : : : : +- CometColumnarExchange + : : : : +- Exchange + : : : : +- HashAggregate : : : : +- 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 + : : : : +- Exchange + : : : : +- 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 : : : +- CometHashAggregate : : : +- CometExchange : : : +- CometHashAggregate @@ -142,31 +139,30 @@ CometNativeColumnarToRow : : : +- 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 + : : : +- Exchange + : : : +- 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 : : +- CometSort : : +- CometProject : : +- CometFilter @@ -209,4 +205,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 127 out of 190 eligible operators (66%). Final plan contains 16 transitions 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/q24a.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q24a.native_datafusion/extended.txt index 03e6809902..8249b86a1e 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q24a.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q24a.native_datafusion/extended.txt @@ -1,96 +1,92 @@ Filter : +- Subquery : +- HashAggregate -: +- CometNativeColumnarToRow -: +- CometColumnarExchange +: +- Exchange +: +- HashAggregate : +- HashAggregate -: +- HashAggregate -: +- CometNativeColumnarToRow -: +- CometColumnarExchange -: +- HashAggregate -: +- Project -: +- BroadcastHashJoin [COMET: Comet is not compatible with Spark for case conversion in locale-specific cases. Set spark.comet.caseConversion.enabled=true to enable it anyway.] -: :- CometNativeColumnarToRow -: : +- CometProject -: : +- CometBroadcastHashJoin -: : :- CometProject -: : : +- CometBroadcastHashJoin -: : : :- CometProject -: : : : +- CometBroadcastHashJoin -: : : : :- CometProject -: : : : : +- CometSortMergeJoin -: : : : : :- CometSort -: : : : : : +- CometExchange -: : : : : : +- CometProject -: : : : : : +- CometFilter -: : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales -: : : : : +- CometSort -: : : : : +- CometExchange -: : : : : +- CometProject -: : : : : +- CometFilter -: : : : : +- CometNativeScan parquet spark_catalog.default.store_returns -: : : : +- CometBroadcastExchange -: : : : +- CometProject -: : : : +- CometFilter -: : : : +- CometNativeScan parquet spark_catalog.default.store -: : : +- CometBroadcastExchange -: : : +- CometProject -: : : +- CometFilter -: : : +- CometNativeScan parquet spark_catalog.default.item -: : +- CometBroadcastExchange -: : +- CometProject -: : +- CometFilter -: : +- CometNativeScan parquet spark_catalog.default.customer -: +- BroadcastExchange -: +- CometNativeColumnarToRow -: +- CometProject -: +- CometFilter -: +- CometNativeScan parquet spark_catalog.default.customer_address +: +- Exchange +: +- HashAggregate +: +- Project +: +- BroadcastHashJoin [COMET: Comet is not compatible with Spark for case conversion in locale-specific cases. Set spark.comet.caseConversion.enabled=true to enable it anyway.] +: :- CometNativeColumnarToRow +: : +- CometProject +: : +- CometBroadcastHashJoin +: : :- CometProject +: : : +- CometBroadcastHashJoin +: : : :- CometProject +: : : : +- CometBroadcastHashJoin +: : : : :- CometProject +: : : : : +- CometSortMergeJoin +: : : : : :- CometSort +: : : : : : +- CometExchange +: : : : : : +- CometProject +: : : : : : +- CometFilter +: : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales +: : : : : +- CometSort +: : : : : +- CometExchange +: : : : : +- CometProject +: : : : : +- CometFilter +: : : : : +- CometNativeScan parquet spark_catalog.default.store_returns +: : : : +- CometBroadcastExchange +: : : : +- CometProject +: : : : +- CometFilter +: : : : +- CometNativeScan parquet spark_catalog.default.store +: : : +- CometBroadcastExchange +: : : +- CometProject +: : : +- CometFilter +: : : +- CometNativeScan parquet spark_catalog.default.item +: : +- CometBroadcastExchange +: : +- CometProject +: : +- CometFilter +: : +- CometNativeScan parquet spark_catalog.default.customer +: +- BroadcastExchange +: +- CometNativeColumnarToRow +: +- CometProject +: +- CometFilter +: +- CometNativeScan parquet spark_catalog.default.customer_address +- HashAggregate - +- CometNativeColumnarToRow - +- CometColumnarExchange + +- Exchange + +- HashAggregate +- HashAggregate - +- HashAggregate - +- CometNativeColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Project - +- BroadcastHashJoin [COMET: Comet is not compatible with Spark for case conversion in locale-specific cases. Set spark.comet.caseConversion.enabled=true to enable it anyway.] - :- CometNativeColumnarToRow - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometSortMergeJoin - : : : : :- CometSort - : : : : : +- CometExchange - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : : +- CometSort - : : : : +- CometExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.store_returns - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.store - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.item - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.customer - +- BroadcastExchange - +- CometNativeColumnarToRow - +- CometProject - +- CometFilter - +- CometNativeScan parquet spark_catalog.default.customer_address + +- Exchange + +- HashAggregate + +- Project + +- BroadcastHashJoin [COMET: Comet is not compatible with Spark for case conversion in locale-specific cases. Set spark.comet.caseConversion.enabled=true to enable it anyway.] + :- CometNativeColumnarToRow + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometSortMergeJoin + : : : : :- CometSort + : : : : : +- CometExchange + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : +- CometSort + : : : : +- CometExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.store_returns + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.store + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.item + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.customer + +- BroadcastExchange + +- CometNativeColumnarToRow + +- CometProject + +- CometFilter + +- CometNativeScan parquet spark_catalog.default.customer_address -Comet accelerated 70 out of 86 eligible operators (81%). Final plan contains 8 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 66 out of 86 eligible operators (76%). 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/q24a.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q24a.native_iceberg_compat/extended.txt index 36748b40ea..fa983a36da 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q24a.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q24a.native_iceberg_compat/extended.txt @@ -1,96 +1,92 @@ Filter : +- Subquery : +- HashAggregate -: +- CometNativeColumnarToRow -: +- CometColumnarExchange +: +- Exchange +: +- HashAggregate : +- HashAggregate -: +- HashAggregate -: +- CometNativeColumnarToRow -: +- CometColumnarExchange -: +- HashAggregate -: +- Project -: +- BroadcastHashJoin [COMET: Comet is not compatible with Spark for case conversion in locale-specific cases. Set spark.comet.caseConversion.enabled=true to enable it anyway.] -: :- CometNativeColumnarToRow -: : +- CometProject -: : +- CometBroadcastHashJoin -: : :- CometProject -: : : +- CometBroadcastHashJoin -: : : :- CometProject -: : : : +- CometBroadcastHashJoin -: : : : :- CometProject -: : : : : +- CometSortMergeJoin -: : : : : :- CometSort -: : : : : : +- CometExchange -: : : : : : +- CometProject -: : : : : : +- CometFilter -: : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -: : : : : +- CometSort -: : : : : +- CometExchange -: : : : : +- CometProject -: : : : : +- CometFilter -: : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns -: : : : +- CometBroadcastExchange -: : : : +- CometProject -: : : : +- CometFilter -: : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store -: : : +- CometBroadcastExchange -: : : +- CometProject -: : : +- CometFilter -: : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item -: : +- CometBroadcastExchange -: : +- CometProject -: : +- CometFilter -: : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer -: +- BroadcastExchange -: +- CometNativeColumnarToRow -: +- CometProject -: +- CometFilter -: +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address +: +- Exchange +: +- HashAggregate +: +- Project +: +- BroadcastHashJoin [COMET: Comet is not compatible with Spark for case conversion in locale-specific cases. Set spark.comet.caseConversion.enabled=true to enable it anyway.] +: :- CometNativeColumnarToRow +: : +- CometProject +: : +- CometBroadcastHashJoin +: : :- CometProject +: : : +- CometBroadcastHashJoin +: : : :- CometProject +: : : : +- CometBroadcastHashJoin +: : : : :- CometProject +: : : : : +- CometSortMergeJoin +: : : : : :- CometSort +: : : : : : +- CometExchange +: : : : : : +- CometProject +: : : : : : +- CometFilter +: : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales +: : : : : +- CometSort +: : : : : +- CometExchange +: : : : : +- CometProject +: : : : : +- CometFilter +: : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns +: : : : +- CometBroadcastExchange +: : : : +- CometProject +: : : : +- CometFilter +: : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store +: : : +- CometBroadcastExchange +: : : +- CometProject +: : : +- CometFilter +: : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item +: : +- CometBroadcastExchange +: : +- CometProject +: : +- CometFilter +: : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer +: +- BroadcastExchange +: +- CometNativeColumnarToRow +: +- CometProject +: +- CometFilter +: +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address +- HashAggregate - +- CometNativeColumnarToRow - +- CometColumnarExchange + +- Exchange + +- HashAggregate +- HashAggregate - +- HashAggregate - +- CometNativeColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Project - +- BroadcastHashJoin [COMET: Comet is not compatible with Spark for case conversion in locale-specific cases. Set spark.comet.caseConversion.enabled=true to enable it anyway.] - :- CometNativeColumnarToRow - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometSortMergeJoin - : : : : :- CometSort - : : : : : +- CometExchange - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : +- CometSort - : : : : +- CometExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer - +- BroadcastExchange - +- CometNativeColumnarToRow - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address + +- Exchange + +- HashAggregate + +- Project + +- BroadcastHashJoin [COMET: Comet is not compatible with Spark for case conversion in locale-specific cases. Set spark.comet.caseConversion.enabled=true to enable it anyway.] + :- CometNativeColumnarToRow + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometSortMergeJoin + : : : : :- CometSort + : : : : : +- CometExchange + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales + : : : : +- CometSort + : : : : +- CometExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer + +- BroadcastExchange + +- CometNativeColumnarToRow + +- CometProject + +- CometFilter + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address -Comet accelerated 70 out of 86 eligible operators (81%). Final plan contains 8 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 66 out of 86 eligible operators (76%). 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/q24b.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q24b.native_datafusion/extended.txt index 03e6809902..8249b86a1e 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q24b.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q24b.native_datafusion/extended.txt @@ -1,96 +1,92 @@ Filter : +- Subquery : +- HashAggregate -: +- CometNativeColumnarToRow -: +- CometColumnarExchange +: +- Exchange +: +- HashAggregate : +- HashAggregate -: +- HashAggregate -: +- CometNativeColumnarToRow -: +- CometColumnarExchange -: +- HashAggregate -: +- Project -: +- BroadcastHashJoin [COMET: Comet is not compatible with Spark for case conversion in locale-specific cases. Set spark.comet.caseConversion.enabled=true to enable it anyway.] -: :- CometNativeColumnarToRow -: : +- CometProject -: : +- CometBroadcastHashJoin -: : :- CometProject -: : : +- CometBroadcastHashJoin -: : : :- CometProject -: : : : +- CometBroadcastHashJoin -: : : : :- CometProject -: : : : : +- CometSortMergeJoin -: : : : : :- CometSort -: : : : : : +- CometExchange -: : : : : : +- CometProject -: : : : : : +- CometFilter -: : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales -: : : : : +- CometSort -: : : : : +- CometExchange -: : : : : +- CometProject -: : : : : +- CometFilter -: : : : : +- CometNativeScan parquet spark_catalog.default.store_returns -: : : : +- CometBroadcastExchange -: : : : +- CometProject -: : : : +- CometFilter -: : : : +- CometNativeScan parquet spark_catalog.default.store -: : : +- CometBroadcastExchange -: : : +- CometProject -: : : +- CometFilter -: : : +- CometNativeScan parquet spark_catalog.default.item -: : +- CometBroadcastExchange -: : +- CometProject -: : +- CometFilter -: : +- CometNativeScan parquet spark_catalog.default.customer -: +- BroadcastExchange -: +- CometNativeColumnarToRow -: +- CometProject -: +- CometFilter -: +- CometNativeScan parquet spark_catalog.default.customer_address +: +- Exchange +: +- HashAggregate +: +- Project +: +- BroadcastHashJoin [COMET: Comet is not compatible with Spark for case conversion in locale-specific cases. Set spark.comet.caseConversion.enabled=true to enable it anyway.] +: :- CometNativeColumnarToRow +: : +- CometProject +: : +- CometBroadcastHashJoin +: : :- CometProject +: : : +- CometBroadcastHashJoin +: : : :- CometProject +: : : : +- CometBroadcastHashJoin +: : : : :- CometProject +: : : : : +- CometSortMergeJoin +: : : : : :- CometSort +: : : : : : +- CometExchange +: : : : : : +- CometProject +: : : : : : +- CometFilter +: : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales +: : : : : +- CometSort +: : : : : +- CometExchange +: : : : : +- CometProject +: : : : : +- CometFilter +: : : : : +- CometNativeScan parquet spark_catalog.default.store_returns +: : : : +- CometBroadcastExchange +: : : : +- CometProject +: : : : +- CometFilter +: : : : +- CometNativeScan parquet spark_catalog.default.store +: : : +- CometBroadcastExchange +: : : +- CometProject +: : : +- CometFilter +: : : +- CometNativeScan parquet spark_catalog.default.item +: : +- CometBroadcastExchange +: : +- CometProject +: : +- CometFilter +: : +- CometNativeScan parquet spark_catalog.default.customer +: +- BroadcastExchange +: +- CometNativeColumnarToRow +: +- CometProject +: +- CometFilter +: +- CometNativeScan parquet spark_catalog.default.customer_address +- HashAggregate - +- CometNativeColumnarToRow - +- CometColumnarExchange + +- Exchange + +- HashAggregate +- HashAggregate - +- HashAggregate - +- CometNativeColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Project - +- BroadcastHashJoin [COMET: Comet is not compatible with Spark for case conversion in locale-specific cases. Set spark.comet.caseConversion.enabled=true to enable it anyway.] - :- CometNativeColumnarToRow - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometSortMergeJoin - : : : : :- CometSort - : : : : : +- CometExchange - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : : +- CometSort - : : : : +- CometExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.store_returns - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.store - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.item - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.customer - +- BroadcastExchange - +- CometNativeColumnarToRow - +- CometProject - +- CometFilter - +- CometNativeScan parquet spark_catalog.default.customer_address + +- Exchange + +- HashAggregate + +- Project + +- BroadcastHashJoin [COMET: Comet is not compatible with Spark for case conversion in locale-specific cases. Set spark.comet.caseConversion.enabled=true to enable it anyway.] + :- CometNativeColumnarToRow + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometSortMergeJoin + : : : : :- CometSort + : : : : : +- CometExchange + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : +- CometSort + : : : : +- CometExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.store_returns + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.store + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.item + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.customer + +- BroadcastExchange + +- CometNativeColumnarToRow + +- CometProject + +- CometFilter + +- CometNativeScan parquet spark_catalog.default.customer_address -Comet accelerated 70 out of 86 eligible operators (81%). Final plan contains 8 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 66 out of 86 eligible operators (76%). 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/q24b.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q24b.native_iceberg_compat/extended.txt index 36748b40ea..fa983a36da 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q24b.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q24b.native_iceberg_compat/extended.txt @@ -1,96 +1,92 @@ Filter : +- Subquery : +- HashAggregate -: +- CometNativeColumnarToRow -: +- CometColumnarExchange +: +- Exchange +: +- HashAggregate : +- HashAggregate -: +- HashAggregate -: +- CometNativeColumnarToRow -: +- CometColumnarExchange -: +- HashAggregate -: +- Project -: +- BroadcastHashJoin [COMET: Comet is not compatible with Spark for case conversion in locale-specific cases. Set spark.comet.caseConversion.enabled=true to enable it anyway.] -: :- CometNativeColumnarToRow -: : +- CometProject -: : +- CometBroadcastHashJoin -: : :- CometProject -: : : +- CometBroadcastHashJoin -: : : :- CometProject -: : : : +- CometBroadcastHashJoin -: : : : :- CometProject -: : : : : +- CometSortMergeJoin -: : : : : :- CometSort -: : : : : : +- CometExchange -: : : : : : +- CometProject -: : : : : : +- CometFilter -: : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -: : : : : +- CometSort -: : : : : +- CometExchange -: : : : : +- CometProject -: : : : : +- CometFilter -: : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns -: : : : +- CometBroadcastExchange -: : : : +- CometProject -: : : : +- CometFilter -: : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store -: : : +- CometBroadcastExchange -: : : +- CometProject -: : : +- CometFilter -: : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item -: : +- CometBroadcastExchange -: : +- CometProject -: : +- CometFilter -: : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer -: +- BroadcastExchange -: +- CometNativeColumnarToRow -: +- CometProject -: +- CometFilter -: +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address +: +- Exchange +: +- HashAggregate +: +- Project +: +- BroadcastHashJoin [COMET: Comet is not compatible with Spark for case conversion in locale-specific cases. Set spark.comet.caseConversion.enabled=true to enable it anyway.] +: :- CometNativeColumnarToRow +: : +- CometProject +: : +- CometBroadcastHashJoin +: : :- CometProject +: : : +- CometBroadcastHashJoin +: : : :- CometProject +: : : : +- CometBroadcastHashJoin +: : : : :- CometProject +: : : : : +- CometSortMergeJoin +: : : : : :- CometSort +: : : : : : +- CometExchange +: : : : : : +- CometProject +: : : : : : +- CometFilter +: : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales +: : : : : +- CometSort +: : : : : +- CometExchange +: : : : : +- CometProject +: : : : : +- CometFilter +: : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns +: : : : +- CometBroadcastExchange +: : : : +- CometProject +: : : : +- CometFilter +: : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store +: : : +- CometBroadcastExchange +: : : +- CometProject +: : : +- CometFilter +: : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item +: : +- CometBroadcastExchange +: : +- CometProject +: : +- CometFilter +: : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer +: +- BroadcastExchange +: +- CometNativeColumnarToRow +: +- CometProject +: +- CometFilter +: +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address +- HashAggregate - +- CometNativeColumnarToRow - +- CometColumnarExchange + +- Exchange + +- HashAggregate +- HashAggregate - +- HashAggregate - +- CometNativeColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Project - +- BroadcastHashJoin [COMET: Comet is not compatible with Spark for case conversion in locale-specific cases. Set spark.comet.caseConversion.enabled=true to enable it anyway.] - :- CometNativeColumnarToRow - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometSortMergeJoin - : : : : :- CometSort - : : : : : +- CometExchange - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : +- CometSort - : : : : +- CometExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer - +- BroadcastExchange - +- CometNativeColumnarToRow - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address + +- Exchange + +- HashAggregate + +- Project + +- BroadcastHashJoin [COMET: Comet is not compatible with Spark for case conversion in locale-specific cases. Set spark.comet.caseConversion.enabled=true to enable it anyway.] + :- CometNativeColumnarToRow + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometSortMergeJoin + : : : : :- CometSort + : : : : : +- CometExchange + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales + : : : : +- CometSort + : : : : +- CometExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer + +- BroadcastExchange + +- CometNativeColumnarToRow + +- CometProject + +- CometFilter + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address -Comet accelerated 70 out of 86 eligible operators (81%). Final plan contains 8 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 66 out of 86 eligible operators (76%). 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/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..81047a7757 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,69 @@ 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 + +- Exchange + +- 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 -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 21 out of 57 eligible operators (36%). 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/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..9a298d216c 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,43 @@ 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 + +- Exchange + +- 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 -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 15 out of 35 eligible operators (42%). 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/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..87d8460ec7 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,44 @@ 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 + +- Exchange + +- 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 -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 15 out of 36 eligible operators (41%). 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/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..98d6e2d24e 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,74 @@ 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 + +- Exchange + +- 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 -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 24 out of 61 eligible operators (39%). 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/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..659f71490a 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 @@ -7,60 +7,57 @@ TakeOrderedAndProject : : +- 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 + : : : +- Exchange + : : : +- 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 + : : +- Exchange + : : +- HashAggregate : : +- 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 + : : +- Exchange + : : +- 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 @@ -72,4 +69,4 @@ TakeOrderedAndProject +- 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 21 out of 61 eligible operators (34%). 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/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..9b960f1ab1 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 @@ -10,141 +10,135 @@ CometNativeColumnarToRow : : : +- 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 + : : : : : +- Exchange + : : : : : +- 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 + : : : : +- Exchange + : : : : +- 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 + : : : +- Exchange + : : : +- 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 + : : +- Exchange + : : +- 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 + : +- Exchange + : +- 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 + +- Exchange + +- 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 -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 32 out of 120 eligible operators (26%). 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-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..69d5009252 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,46 @@ 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 ++- Exchange + +- 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 + : +- Exchange + : +- 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 -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 12 out of 38 eligible operators (31%). 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/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..b237976296 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,108 @@ 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 + +- Exchange + +- HashAggregate + +- Union + :- HashAggregate + : +- Exchange + : +- 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 + : +- Exchange + : +- 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 + +- Exchange + +- 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 -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 42 out of 93 eligible operators (45%). 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/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..575c853e76 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 @@ -5,43 +5,42 @@ CometNativeColumnarToRow +- 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 + : +- Exchange + : +- 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 -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 17 out of 37 eligible operators (45%). 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/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..c8a9740a02 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 @@ -1,67 +1,66 @@ TakeOrderedAndProject +- HashAggregate - +- CometNativeColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- 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 - : : : : +- 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 - : : : +- 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 - : : +- 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 + +- Exchange + +- HashAggregate + +- Project + +- BroadcastHashJoin + :- Project + : +- BroadcastHashJoin + : :- 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 + : : : : +- 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 + : : : +- 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 + : : +- 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 -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 20 out of 54 eligible operators (37%). 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/q35.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q35.native_iceberg_compat/extended.txt index c5125b1861..72960fc267 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q35.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q35.native_iceberg_compat/extended.txt @@ -1,63 +1,62 @@ TakeOrderedAndProject +- HashAggregate - +- CometNativeColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- Project - : : +- Filter - : : +- BroadcastHashJoin - : : :- BroadcastHashJoin [COMET: Unsupported join type ExistenceJoin(exists#1)] - : : : :- CometNativeColumnarToRow - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : +- ReusedSubquery - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : +- ReusedSubquery - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- CometNativeColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address - +- BroadcastExchange - +- CometNativeColumnarToRow - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics + +- Exchange + +- HashAggregate + +- Project + +- BroadcastHashJoin + :- Project + : +- BroadcastHashJoin + : :- Project + : : +- Filter + : : +- BroadcastHashJoin + : : :- BroadcastHashJoin [COMET: Unsupported join type ExistenceJoin(exists#1)] + : : : :- CometNativeColumnarToRow + : : : : +- CometBroadcastHashJoin + : : : : :- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometNativeColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : +- BroadcastExchange + : : : +- CometNativeColumnarToRow + : : : +- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales + : : : : +- ReusedSubquery + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- BroadcastExchange + : : +- CometNativeColumnarToRow + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales + : : : +- ReusedSubquery + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- BroadcastExchange + : +- CometNativeColumnarToRow + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address + +- BroadcastExchange + +- CometNativeColumnarToRow + +- CometProject + +- CometFilter + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics -Comet accelerated 35 out of 54 eligible operators (64%). Final plan contains 7 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 34 out of 54 eligible operators (62%). 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 aa40ed68cc..7b401d7eb8 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 @@ -5,39 +5,38 @@ TakeOrderedAndProject +- 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 + +- Exchange + +- 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 -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 14 out of 34 eligible operators (41%). 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/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..c38a41ba07 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 @@ -5,71 +5,69 @@ CometNativeColumnarToRow :- 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 + : +- Exchange + : +- 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 + +- Exchange + +- 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 -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 22 out of 60 eligible operators (36%). 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/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..c38a41ba07 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 @@ -5,71 +5,69 @@ CometNativeColumnarToRow :- 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 + : +- Exchange + : +- 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 + +- Exchange + +- 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 -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 22 out of 60 eligible operators (36%). 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/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..d28d212f33 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 @@ -10,145 +10,139 @@ TakeOrderedAndProject : : : :- 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 + : : : : : +- Exchange + : : : : : +- 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 + : : : : +- Exchange + : : : : +- 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 + : : : +- Exchange + : : : +- 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 + : : +- Exchange + : : +- 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 + : +- Exchange + : +- 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 + +- Exchange + +- 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 -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 34 out of 126 eligible operators (26%). Final plan contains 20 transitions 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..4f2175cf7b 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 @@ -1,51 +1,50 @@ TakeOrderedAndProject +- HashAggregate - +- CometNativeColumnarToRow - +- CometColumnarExchange - +- 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 - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.item - +- BroadcastExchange - +- CometNativeColumnarToRow - +- CometProject - +- CometFilter - +- CometNativeScan parquet spark_catalog.default.item + +- Exchange + +- 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 + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.item + +- BroadcastExchange + +- CometNativeColumnarToRow + +- CometProject + +- 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 17 out of 41 eligible operators (41%). 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/q45.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q45.native_iceberg_compat/extended.txt index afdf8d6633..f8c94605ab 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q45.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q45.native_iceberg_compat/extended.txt @@ -1,47 +1,46 @@ TakeOrderedAndProject +- HashAggregate - +- CometNativeColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Project - +- Filter - +- BroadcastHashJoin [COMET: Unsupported join type ExistenceJoin(exists#1)] - :- CometNativeColumnarToRow - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - +- BroadcastExchange - +- CometNativeColumnarToRow - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + +- Exchange + +- HashAggregate + +- Project + +- Filter + +- BroadcastHashJoin [COMET: Unsupported join type ExistenceJoin(exists#1)] + :- CometNativeColumnarToRow + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometNativeColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + +- BroadcastExchange + +- CometNativeColumnarToRow + +- CometProject + +- CometFilter + +- CometScan [native_iceberg_compat] 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 31 out of 41 eligible operators (75%). 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 cebcd85144..7aaa58fcbf 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 @@ -4,45 +4,44 @@ TakeOrderedAndProject :- 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 + : : +- Exchange + : : +- 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 @@ -53,4 +52,4 @@ TakeOrderedAndProject +- 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 19 out of 45 eligible operators (42%). 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/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..e347d0f433 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 @@ -12,36 +12,35 @@ TakeOrderedAndProject : : +- 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 + : : +- Exchange + : : +- 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 : +- 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).] @@ -49,36 +48,35 @@ TakeOrderedAndProject : +- 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 + : +- Exchange + : +- 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 +- 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).] @@ -86,35 +84,34 @@ TakeOrderedAndProject +- 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 + +- Exchange + +- 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 -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 33 out of 97 eligible operators (34%). 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-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..411a47f8ed 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,41 @@ 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 ++- Exchange + +- 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 -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 14 out of 33 eligible operators (42%). 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/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..d19eb04caf 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 @@ -13,33 +13,32 @@ 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 + : +- Exchange + : +- 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 :- Project : +- Filter : +- Window @@ -49,28 +48,27 @@ 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 + : +- Exchange + : +- 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 +- Project +- Filter +- Window @@ -80,27 +78,26 @@ 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 + +- Exchange + +- 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 -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 30 out of 87 eligible operators (34%). 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-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..c9204f8e91 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,102 @@ 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 + +- Exchange + +- HashAggregate + +- Expand + +- Union + :- HashAggregate + : +- Exchange + : +- 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 + : +- Exchange + : +- 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 + +- Exchange + +- 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 -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 24 out of 86 eligible operators (27%). 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-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..d0b7b01cee 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,41 @@ 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 + +- Exchange + +- 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 -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 13 out of 33 eligible operators (39%). 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/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..ee122e3dcd 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 @@ -14,25 +14,24 @@ TakeOrderedAndProject : +- 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 + : +- Exchange + : +- 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 +- CometSort +- CometColumnarExchange +- Project @@ -41,19 +40,18 @@ TakeOrderedAndProject +- 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 + +- Exchange + +- 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 -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 21 out of 47 eligible operators (44%). 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/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..9b3651b3fb 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 @@ -6,37 +6,36 @@ TakeOrderedAndProject +- 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 + +- Exchange + +- 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 -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 13 out of 33 eligible operators (39%). 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/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..bb8b3dfe0d 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,118 @@ TakeOrderedAndProject +- HashAggregate - +- CometNativeColumnarToRow - +- CometColumnarExchange + +- Exchange + +- HashAggregate +- 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 + +- Exchange + +- 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 + +- 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 49 out of 100 eligible operators (49%). Final plan contains 16 transitions 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..4babc2254d 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,111 @@ 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 + +- Exchange + +- HashAggregate + +- Union + :- HashAggregate + : +- Exchange + : +- 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 + : +- Exchange + : +- 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 + +- Exchange + +- 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 -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 45 out of 96 eligible operators (46%). 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/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..50dc55f674 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 @@ -12,36 +12,35 @@ TakeOrderedAndProject : : +- 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 + : : +- Exchange + : : +- 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 : +- 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).] @@ -49,36 +48,35 @@ TakeOrderedAndProject : +- 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 + : +- Exchange + : +- 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 +- 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).] @@ -86,35 +84,34 @@ TakeOrderedAndProject +- 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 + +- Exchange + +- 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 -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 33 out of 97 eligible operators (34%). 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-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..18f059b348 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 @@ -5,123 +5,120 @@ TakeOrderedAndProject : +- 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 + : : +- Exchange + : : +- 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 + : +- Exchange + : +- 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 + +- Exchange + +- 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 -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 53 out of 108 eligible operators (49%). 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-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..bdf9399f8b 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,69 @@ 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 + +- Exchange + +- 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 -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 38 out of 60 eligible operators (63%). 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/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..4babc2254d 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,111 @@ 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 + +- Exchange + +- HashAggregate + +- Union + :- HashAggregate + : +- Exchange + : +- 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 + : +- Exchange + : +- 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 + +- Exchange + +- 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 -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 45 out of 96 eligible operators (46%). 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/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..da7b7fc2f0 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,99 @@ 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 + : +- Exchange + : +- 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 +- 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 + +- Exchange + +- 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 -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 34 out of 83 eligible operators (40%). 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-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..9b3651b3fb 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 @@ -6,37 +6,36 @@ TakeOrderedAndProject +- 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 + +- Exchange + +- 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 -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 13 out of 33 eligible operators (39%). 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/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..f059866684 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 @@ -11,25 +11,24 @@ TakeOrderedAndProject : : +- 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 + : : +- Exchange + : : +- 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 @@ -38,23 +37,21 @@ TakeOrderedAndProject +- BroadcastExchange +- Filter +- HashAggregate - +- CometNativeColumnarToRow - +- CometColumnarExchange + +- Exchange + +- HashAggregate +- 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 + +- Exchange + +- 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 -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 14 out of 48 eligible operators (29%). 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/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..b586928ede 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,79 @@ 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 + +- Exchange + +- HashAggregate + +- Union + :- HashAggregate + : +- Exchange + : +- 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 + +- Exchange + +- 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 -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 24 out of 66 eligible operators (36%). 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/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..611eb3c40f 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 @@ -8,39 +8,38 @@ TakeOrderedAndProject +- 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 + +- Exchange + +- 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 -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 14 out of 37 eligible operators (37%). 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/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..7aaa58fcbf 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 @@ -4,45 +4,44 @@ TakeOrderedAndProject :- 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 + : : +- Exchange + : : +- 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 @@ -53,4 +52,4 @@ TakeOrderedAndProject +- 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 19 out of 45 eligible operators (42%). 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/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..c8bb5a4bde 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 @@ -1,66 +1,65 @@ TakeOrderedAndProject +- HashAggregate - +- CometNativeColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Project - +- BroadcastHashJoin - :- Project - : +- 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 - : : : : +- 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 - : : : +- 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 - : : +- 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 + +- Exchange + +- HashAggregate + +- Project + +- BroadcastHashJoin + :- Project + : +- 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 + : : : : +- 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 + : : : +- 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 + : : +- 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 -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 20 out of 53 eligible operators (37%). 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/q69.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q69.native_iceberg_compat/extended.txt index bf624b5ce3..726fb6b283 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q69.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q69.native_iceberg_compat/extended.txt @@ -1,62 +1,61 @@ TakeOrderedAndProject +- HashAggregate - +- CometNativeColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- BroadcastHashJoin [COMET: BuildRight with LeftAnti is not supported] - : : : :- CometNativeColumnarToRow - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : +- ReusedSubquery - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : +- ReusedSubquery - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- CometNativeColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address - +- BroadcastExchange - +- CometNativeColumnarToRow - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics + +- Exchange + +- HashAggregate + +- Project + +- BroadcastHashJoin + :- Project + : +- BroadcastHashJoin + : :- Project + : : +- BroadcastHashJoin + : : :- BroadcastHashJoin [COMET: BuildRight with LeftAnti is not supported] + : : : :- CometNativeColumnarToRow + : : : : +- CometBroadcastHashJoin + : : : : :- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometNativeColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : +- BroadcastExchange + : : : +- CometNativeColumnarToRow + : : : +- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales + : : : : +- ReusedSubquery + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- BroadcastExchange + : : +- CometNativeColumnarToRow + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales + : : : +- ReusedSubquery + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- BroadcastExchange + : +- CometNativeColumnarToRow + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address + +- BroadcastExchange + +- CometNativeColumnarToRow + +- CometProject + +- CometFilter + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics -Comet accelerated 35 out of 53 eligible operators (66%). Final plan contains 7 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 34 out of 53 eligible operators (64%). 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 b692dfa9be..60c0417071 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,43 @@ 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 + +- Exchange + +- 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 -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 15 out of 35 eligible operators (42%). 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 f222353c17..1e004dfc43 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 @@ -5,61 +5,59 @@ TakeOrderedAndProject +- CometSort +- CometColumnarExchange +- HashAggregate - +- CometNativeColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- 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 - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.date_dim - +- BroadcastExchange - +- Project - +- BroadcastHashJoin - :- CometNativeColumnarToRow - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.store - +- BroadcastExchange - +- 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 + +- Exchange + +- HashAggregate + +- 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 + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.date_dim + +- BroadcastExchange + +- Project + +- BroadcastHashJoin + :- CometNativeColumnarToRow + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.store + +- BroadcastExchange + +- Project + +- Filter + +- Window + +- WindowGroupLimit + +- Sort + +- HashAggregate + +- Exchange + +- 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 -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 16 out of 53 eligible operators (30%). 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/q70.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q70.native_iceberg_compat/extended.txt index 95077efbc5..1ce50f8385 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q70.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q70.native_iceberg_compat/extended.txt @@ -5,57 +5,56 @@ TakeOrderedAndProject +- CometSort +- CometColumnarExchange +- HashAggregate - +- CometNativeColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Expand - +- Project - +- BroadcastHashJoin - :- CometNativeColumnarToRow - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - +- BroadcastExchange - +- Project - +- BroadcastHashJoin - :- CometNativeColumnarToRow - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - +- BroadcastExchange - +- Project - +- Filter - +- Window - +- WindowGroupLimit [COMET: WindowGroupLimit is not supported] - +- CometNativeColumnarToRow - +- CometSort - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : +- ReusedSubquery - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + +- Exchange + +- HashAggregate + +- Expand + +- Project + +- BroadcastHashJoin + :- CometNativeColumnarToRow + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales + : : +- SubqueryBroadcast + : : +- BroadcastExchange + : : +- CometNativeColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + +- BroadcastExchange + +- Project + +- BroadcastHashJoin + :- CometNativeColumnarToRow + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store + +- BroadcastExchange + +- Project + +- Filter + +- Window + +- WindowGroupLimit [COMET: WindowGroupLimit is not supported] + +- CometNativeColumnarToRow + +- CometSort + +- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales + : : +- ReusedSubquery + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store + +- CometBroadcastExchange + +- CometProject + +- CometFilter + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Comet accelerated 34 out of 53 eligible operators (64%). Final plan contains 6 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 33 out of 53 eligible operators (62%). 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 23844f41f3..9d5665df4d 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 @@ -2,61 +2,60 @@ 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 + +- Exchange + +- 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 -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 20 out of 49 eligible operators (40%). 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/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..575c853e76 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 @@ -5,43 +5,42 @@ CometNativeColumnarToRow +- 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 + : +- Exchange + : +- 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 -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 17 out of 37 eligible operators (45%). 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/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..00d2b3b0d7 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 @@ -6,100 +6,96 @@ TakeOrderedAndProject : :- 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 + : : : +- Exchange + : : : +- 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 + : : +- Exchange + : : +- 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 + : +- Exchange + : +- 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 + +- Exchange + +- 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 -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 24 out of 85 eligible operators (28%). 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-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..e647d8db46 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 @@ -1,135 +1,128 @@ TakeOrderedAndProject +- HashAggregate - +- CometNativeColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Expand - +- Union - :- 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 - : :- 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 + +- Exchange + +- HashAggregate + +- Expand + +- Union + :- Project + : +- BroadcastHashJoin + : :- HashAggregate + : : +- Exchange + : : +- 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 + : +- Exchange + : +- 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 + : :- BroadcastExchange + : : +- HashAggregate + : : +- Exchange + : : +- 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 + : +- Exchange + : +- 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 + : +- Exchange + : +- 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 + +- Exchange + +- 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 -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 29 out of 109 eligible operators (26%). Final plan contains 17 transitions 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_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q77.native_iceberg_compat/extended.txt index 0cfc0a8417..b2653c7068 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q77.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q77.native_iceberg_compat/extended.txt @@ -1,117 +1,116 @@ TakeOrderedAndProject +- HashAggregate - +- CometNativeColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Expand - +- Union - :- CometNativeColumnarToRow - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometHashAggregate - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - : +- CometBroadcastExchange - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns - : : : +- ReusedSubquery - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - :- Project - : +- BroadcastNestedLoopJoin [COMET: BroadcastNestedLoopJoin is not supported] - : :- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometHashAggregate - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : +- ReusedSubquery - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometNativeColumnarToRow - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns - : : +- ReusedSubquery - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - +- CometNativeColumnarToRow - +- CometProject - +- CometBroadcastHashJoin - :- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : +- ReusedSubquery - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_page - +- CometBroadcastExchange - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns - : : +- ReusedSubquery - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - +- CometBroadcastExchange - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_page + +- Exchange + +- HashAggregate + +- Expand + +- Union + :- CometNativeColumnarToRow + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometHashAggregate + : : +- CometExchange + : : +- CometHashAggregate + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometNativeColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store + : +- CometBroadcastExchange + : +- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns + : : : +- ReusedSubquery + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store + :- Project + : +- BroadcastNestedLoopJoin [COMET: BroadcastNestedLoopJoin is not supported] + : :- BroadcastExchange + : : +- CometNativeColumnarToRow + : : +- CometHashAggregate + : : +- CometExchange + : : +- CometHashAggregate + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales + : : : +- ReusedSubquery + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- CometNativeColumnarToRow + : +- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns + : : +- ReusedSubquery + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + +- CometNativeColumnarToRow + +- CometProject + +- CometBroadcastHashJoin + :- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales + : : : +- ReusedSubquery + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_page + +- CometBroadcastExchange + +- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns + : : +- ReusedSubquery + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + +- CometBroadcastExchange + +- CometFilter + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_page -Comet accelerated 94 out of 109 eligible operators (86%). Final plan contains 6 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 93 out of 109 eligible operators (85%). 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/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..76bf23f16f 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 @@ -2,43 +2,42 @@ 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 + : +- Exchange + : +- 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 -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 15 out of 35 eligible operators (42%). 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/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..838592f73b 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,55 @@ 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 + +- Exchange + +- 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 -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 31 out of 48 eligible operators (64%). 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/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..6133951418 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 @@ -7,60 +7,57 @@ TakeOrderedAndProject : : +- 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 + : : : +- Exchange + : : : +- 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 + : : +- Exchange + : : +- HashAggregate : : +- 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 + : : +- Exchange + : : +- 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 @@ -72,4 +69,4 @@ TakeOrderedAndProject +- 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 21 out of 61 eligible operators (34%). 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/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..66a0c08b10 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 @@ -4,113 +4,110 @@ TakeOrderedAndProject :- 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 + : : +- Exchange + : : +- 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 + : +- Exchange + : +- 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 + +- Exchange + +- 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 -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 57 out of 101 eligible operators (56%). 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/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..efab56234e 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,63 @@ 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 + +- Exchange + +- 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 -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 23 out of 52 eligible operators (44%). 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/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..7313f7e979 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 @@ -5,32 +5,31 @@ TakeOrderedAndProject +- 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 + +- Exchange + +- 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 -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 11 out of 28 eligible operators (39%). 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/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..772b0f4512 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 @@ -1,82 +1,81 @@ HashAggregate -+- CometNativeColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Project - +- BroadcastHashJoin - :- 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 - : +- 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 - +- 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 ++- Exchange + +- HashAggregate + +- Project + +- BroadcastHashJoin + :- 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 + : +- 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 + +- 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 -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 27 out of 66 eligible operators (40%). 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/q87.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q87.native_iceberg_compat/extended.txt index 5fddd74768..6218729c98 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q87.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q87.native_iceberg_compat/extended.txt @@ -1,73 +1,72 @@ HashAggregate -+- CometNativeColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Project - +- BroadcastHashJoin - :- BroadcastHashJoin [COMET: BuildRight with LeftAnti is not supported] - : :- CometNativeColumnarToRow - : : +- CometHashAggregate - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer - : +- BroadcastExchange - : +- CometNativeColumnarToRow - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : +- ReusedSubquery - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer - +- BroadcastExchange - +- CometNativeColumnarToRow - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : +- ReusedSubquery - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer ++- Exchange + +- HashAggregate + +- Project + +- BroadcastHashJoin + :- BroadcastHashJoin [COMET: BuildRight with LeftAnti is not supported] + : :- CometNativeColumnarToRow + : : +- CometHashAggregate + : : +- CometExchange + : : +- CometHashAggregate + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometNativeColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer + : +- BroadcastExchange + : +- CometNativeColumnarToRow + : +- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales + : : : +- ReusedSubquery + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer + +- BroadcastExchange + +- CometNativeColumnarToRow + +- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales + : : +- ReusedSubquery + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + +- CometBroadcastExchange + +- CometProject + +- CometFilter + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer -Comet accelerated 55 out of 66 eligible operators (83%). Final plan contains 5 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 54 out of 66 eligible operators (81%). 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 5c7a26057d..9b3651b3fb 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 @@ -6,37 +6,36 @@ TakeOrderedAndProject +- 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 + +- Exchange + +- 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 -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 13 out of 33 eligible operators (39%). 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/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..27bfd61c53 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 @@ -2,58 +2,57 @@ 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 + +- Exchange + +- 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 -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 22 out of 47 eligible operators (46%). 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/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..c0f40501d8 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,46 @@ 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 ++- Exchange + +- 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 + : +- Exchange + : +- 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 -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 12 out of 38 eligible operators (31%). 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/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..9de7969459 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 @@ -8,31 +8,30 @@ 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 + +- Exchange + +- 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 -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 14 out of 29 eligible operators (48%). 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/q1.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q1.native_datafusion/extended.txt index 91b07a7538..a3434fe3ea 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 @@ -7,46 +7,43 @@ TakeOrderedAndProject : : +- 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 + : : : +- Exchange + : : : +- 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 + : : +- Exchange + : : +- HashAggregate : : +- 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 + : : +- Exchange + : : +- 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 @@ -58,4 +55,4 @@ TakeOrderedAndProject +- 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 15 out of 49 eligible operators (30%). 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/q10.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q10.native_datafusion/extended.txt index 40a8ef4a6e..c8a9740a02 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 @@ -1,67 +1,66 @@ TakeOrderedAndProject +- HashAggregate - +- CometNativeColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- 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 - : : : : +- 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 - : : : +- 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 - : : +- 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 + +- Exchange + +- HashAggregate + +- Project + +- BroadcastHashJoin + :- Project + : +- BroadcastHashJoin + : :- 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 + : : : : +- 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 + : : : +- 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 + : : +- 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 -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 20 out of 54 eligible operators (37%). 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/q10.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q10.native_iceberg_compat/extended.txt index c5125b1861..72960fc267 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q10.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q10.native_iceberg_compat/extended.txt @@ -1,63 +1,62 @@ TakeOrderedAndProject +- HashAggregate - +- CometNativeColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- Project - : : +- Filter - : : +- BroadcastHashJoin - : : :- BroadcastHashJoin [COMET: Unsupported join type ExistenceJoin(exists#1)] - : : : :- CometNativeColumnarToRow - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : +- ReusedSubquery - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : +- ReusedSubquery - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- CometNativeColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address - +- BroadcastExchange - +- CometNativeColumnarToRow - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics + +- Exchange + +- HashAggregate + +- Project + +- BroadcastHashJoin + :- Project + : +- BroadcastHashJoin + : :- Project + : : +- Filter + : : +- BroadcastHashJoin + : : :- BroadcastHashJoin [COMET: Unsupported join type ExistenceJoin(exists#1)] + : : : :- CometNativeColumnarToRow + : : : : +- CometBroadcastHashJoin + : : : : :- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometNativeColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : +- BroadcastExchange + : : : +- CometNativeColumnarToRow + : : : +- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales + : : : : +- ReusedSubquery + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- BroadcastExchange + : : +- CometNativeColumnarToRow + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales + : : : +- ReusedSubquery + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- BroadcastExchange + : +- CometNativeColumnarToRow + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address + +- BroadcastExchange + +- CometNativeColumnarToRow + +- CometProject + +- CometFilter + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics -Comet accelerated 35 out of 54 eligible operators (64%). Final plan contains 7 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 34 out of 54 eligible operators (62%). 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 47aa9517cf..a38205d189 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 @@ -7,100 +7,96 @@ TakeOrderedAndProject : : +- 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 + : : : +- Exchange + : : : +- 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 + : : +- Exchange + : : +- 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 + : +- Exchange + : +- 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 + +- Exchange + +- 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 -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 24 out of 86 eligible operators (27%). 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-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..79e4955126 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 @@ -5,31 +5,30 @@ TakeOrderedAndProject +- 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 + +- Exchange + +- 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 -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 11 out of 27 eligible operators (40%). 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/q13.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q13.native_datafusion/extended.txt index 9ff51aab85..886c7fc5fe 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,47 @@ 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 ++- Exchange + +- 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 -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 16 out of 38 eligible operators (42%). 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/q14a.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14a.native_datafusion/extended.txt index 223593e3d0..4ed257661d 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,548 @@ TakeOrderedAndProject +- HashAggregate - +- CometNativeColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Expand - +- Union - :- Project - : +- 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 - : : : : +- 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 - : : +- 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 - : +- 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 + +- Exchange + +- HashAggregate + +- Expand + +- Union + :- Project + : +- Filter + : : +- Subquery + : : +- HashAggregate + : : +- Exchange + : : +- 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 + : +- Exchange + : +- 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 + : : +- ReusedSubquery + : +- HashAggregate + : +- Exchange + : +- 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 + : +- ReusedSubquery + +- HashAggregate + +- Exchange + +- 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 -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 159 out of 458 eligible operators (34%). Final plan contains 88 transitions 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..8d34a30261 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 @@ -3,402 +3,399 @@ TakeOrderedAndProject :- 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 + : : +- Exchange + : : +- 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 + : +- Exchange + : +- 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 + +- Exchange + +- 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 -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 125 out of 333 eligible operators (37%). Final plan contains 66 transitions 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..822dcf6212 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,35 @@ 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 + +- Exchange + +- 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 -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 11 out of 28 eligible operators (39%). 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/q17.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q17.native_datafusion/extended.txt index 15c6bec5dd..81047a7757 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,69 @@ 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 + +- Exchange + +- 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 -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 21 out of 57 eligible operators (36%). 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/q18.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q18.native_datafusion/extended.txt index 53fa7b6342..8b93be4c00 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,57 @@ 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 + +- Exchange + +- 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 -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 20 out of 47 eligible operators (42%). 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/q20.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q20.native_datafusion/extended.txt index 6634edcbed..74b85d1536 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 @@ -5,31 +5,30 @@ TakeOrderedAndProject +- 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 + +- Exchange + +- 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 -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 11 out of 27 eligible operators (40%). 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/q21.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q21.native_datafusion/extended.txt index 407648ea84..d0c6d62f37 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,34 @@ 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 + +- Exchange + +- 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 -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 9 out of 27 eligible operators (33%). 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/q22.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q22.native_datafusion/extended.txt index e1766e64d8..036d0bad77 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,36 @@ 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 + +- Exchange + +- 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 -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 11 out of 29 eligible operators (37%). 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/q23a.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q23a.native_datafusion/extended.txt index 7040e78da1..5aa139bc02 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 @@ -23,65 +23,62 @@ CometNativeColumnarToRow : : : +- 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 + : : : +- Exchange + : : : +- 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 : : +- CometSort : : +- CometProject : : +- CometFilter : : : +- Subquery : : : +- HashAggregate - : : : +- CometNativeColumnarToRow - : : : +- CometColumnarExchange + : : : +- Exchange + : : : +- HashAggregate : : : +- 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 + : : : +- Exchange + : : : +- 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 : : +- CometHashAggregate : : +- CometExchange : : +- CometHashAggregate @@ -112,31 +109,30 @@ CometNativeColumnarToRow : : +- 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 + : : +- Exchange + : : +- 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 : +- CometSort : +- CometProject : +- CometFilter @@ -157,4 +153,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 79 out of 138 eligible operators (57%). Final plan contains 16 transitions 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..68e2df37fc 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 @@ -26,65 +26,62 @@ CometNativeColumnarToRow : : : : +- 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 + : : : : +- Exchange + : : : : +- 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 : : : +- CometSort : : : +- CometProject : : : +- CometFilter : : : : +- Subquery : : : : +- HashAggregate - : : : : +- CometNativeColumnarToRow - : : : : +- CometColumnarExchange + : : : : +- Exchange + : : : : +- HashAggregate : : : : +- 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 + : : : : +- Exchange + : : : : +- 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 : : : +- CometHashAggregate : : : +- CometExchange : : : +- CometHashAggregate @@ -142,31 +139,30 @@ CometNativeColumnarToRow : : : +- 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 + : : : +- Exchange + : : : +- 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 : : +- CometSort : : +- CometProject : : +- CometFilter @@ -209,4 +205,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 127 out of 190 eligible operators (66%). Final plan contains 16 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q24a.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q24a.native_datafusion/extended.txt index 03e6809902..8249b86a1e 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q24a.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q24a.native_datafusion/extended.txt @@ -1,96 +1,92 @@ Filter : +- Subquery : +- HashAggregate -: +- CometNativeColumnarToRow -: +- CometColumnarExchange +: +- Exchange +: +- HashAggregate : +- HashAggregate -: +- HashAggregate -: +- CometNativeColumnarToRow -: +- CometColumnarExchange -: +- HashAggregate -: +- Project -: +- BroadcastHashJoin [COMET: Comet is not compatible with Spark for case conversion in locale-specific cases. Set spark.comet.caseConversion.enabled=true to enable it anyway.] -: :- CometNativeColumnarToRow -: : +- CometProject -: : +- CometBroadcastHashJoin -: : :- CometProject -: : : +- CometBroadcastHashJoin -: : : :- CometProject -: : : : +- CometBroadcastHashJoin -: : : : :- CometProject -: : : : : +- CometSortMergeJoin -: : : : : :- CometSort -: : : : : : +- CometExchange -: : : : : : +- CometProject -: : : : : : +- CometFilter -: : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales -: : : : : +- CometSort -: : : : : +- CometExchange -: : : : : +- CometProject -: : : : : +- CometFilter -: : : : : +- CometNativeScan parquet spark_catalog.default.store_returns -: : : : +- CometBroadcastExchange -: : : : +- CometProject -: : : : +- CometFilter -: : : : +- CometNativeScan parquet spark_catalog.default.store -: : : +- CometBroadcastExchange -: : : +- CometProject -: : : +- CometFilter -: : : +- CometNativeScan parquet spark_catalog.default.item -: : +- CometBroadcastExchange -: : +- CometProject -: : +- CometFilter -: : +- CometNativeScan parquet spark_catalog.default.customer -: +- BroadcastExchange -: +- CometNativeColumnarToRow -: +- CometProject -: +- CometFilter -: +- CometNativeScan parquet spark_catalog.default.customer_address +: +- Exchange +: +- HashAggregate +: +- Project +: +- BroadcastHashJoin [COMET: Comet is not compatible with Spark for case conversion in locale-specific cases. Set spark.comet.caseConversion.enabled=true to enable it anyway.] +: :- CometNativeColumnarToRow +: : +- CometProject +: : +- CometBroadcastHashJoin +: : :- CometProject +: : : +- CometBroadcastHashJoin +: : : :- CometProject +: : : : +- CometBroadcastHashJoin +: : : : :- CometProject +: : : : : +- CometSortMergeJoin +: : : : : :- CometSort +: : : : : : +- CometExchange +: : : : : : +- CometProject +: : : : : : +- CometFilter +: : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales +: : : : : +- CometSort +: : : : : +- CometExchange +: : : : : +- CometProject +: : : : : +- CometFilter +: : : : : +- CometNativeScan parquet spark_catalog.default.store_returns +: : : : +- CometBroadcastExchange +: : : : +- CometProject +: : : : +- CometFilter +: : : : +- CometNativeScan parquet spark_catalog.default.store +: : : +- CometBroadcastExchange +: : : +- CometProject +: : : +- CometFilter +: : : +- CometNativeScan parquet spark_catalog.default.item +: : +- CometBroadcastExchange +: : +- CometProject +: : +- CometFilter +: : +- CometNativeScan parquet spark_catalog.default.customer +: +- BroadcastExchange +: +- CometNativeColumnarToRow +: +- CometProject +: +- CometFilter +: +- CometNativeScan parquet spark_catalog.default.customer_address +- HashAggregate - +- CometNativeColumnarToRow - +- CometColumnarExchange + +- Exchange + +- HashAggregate +- HashAggregate - +- HashAggregate - +- CometNativeColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Project - +- BroadcastHashJoin [COMET: Comet is not compatible with Spark for case conversion in locale-specific cases. Set spark.comet.caseConversion.enabled=true to enable it anyway.] - :- CometNativeColumnarToRow - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometSortMergeJoin - : : : : :- CometSort - : : : : : +- CometExchange - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : : +- CometSort - : : : : +- CometExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.store_returns - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.store - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.item - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.customer - +- BroadcastExchange - +- CometNativeColumnarToRow - +- CometProject - +- CometFilter - +- CometNativeScan parquet spark_catalog.default.customer_address + +- Exchange + +- HashAggregate + +- Project + +- BroadcastHashJoin [COMET: Comet is not compatible with Spark for case conversion in locale-specific cases. Set spark.comet.caseConversion.enabled=true to enable it anyway.] + :- CometNativeColumnarToRow + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometSortMergeJoin + : : : : :- CometSort + : : : : : +- CometExchange + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : +- CometSort + : : : : +- CometExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.store_returns + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.store + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.item + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.customer + +- BroadcastExchange + +- CometNativeColumnarToRow + +- CometProject + +- CometFilter + +- CometNativeScan parquet spark_catalog.default.customer_address -Comet accelerated 70 out of 86 eligible operators (81%). Final plan contains 8 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 66 out of 86 eligible operators (76%). 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/q24a.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q24a.native_iceberg_compat/extended.txt index 36748b40ea..fa983a36da 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q24a.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q24a.native_iceberg_compat/extended.txt @@ -1,96 +1,92 @@ Filter : +- Subquery : +- HashAggregate -: +- CometNativeColumnarToRow -: +- CometColumnarExchange +: +- Exchange +: +- HashAggregate : +- HashAggregate -: +- HashAggregate -: +- CometNativeColumnarToRow -: +- CometColumnarExchange -: +- HashAggregate -: +- Project -: +- BroadcastHashJoin [COMET: Comet is not compatible with Spark for case conversion in locale-specific cases. Set spark.comet.caseConversion.enabled=true to enable it anyway.] -: :- CometNativeColumnarToRow -: : +- CometProject -: : +- CometBroadcastHashJoin -: : :- CometProject -: : : +- CometBroadcastHashJoin -: : : :- CometProject -: : : : +- CometBroadcastHashJoin -: : : : :- CometProject -: : : : : +- CometSortMergeJoin -: : : : : :- CometSort -: : : : : : +- CometExchange -: : : : : : +- CometProject -: : : : : : +- CometFilter -: : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -: : : : : +- CometSort -: : : : : +- CometExchange -: : : : : +- CometProject -: : : : : +- CometFilter -: : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns -: : : : +- CometBroadcastExchange -: : : : +- CometProject -: : : : +- CometFilter -: : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store -: : : +- CometBroadcastExchange -: : : +- CometProject -: : : +- CometFilter -: : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item -: : +- CometBroadcastExchange -: : +- CometProject -: : +- CometFilter -: : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer -: +- BroadcastExchange -: +- CometNativeColumnarToRow -: +- CometProject -: +- CometFilter -: +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address +: +- Exchange +: +- HashAggregate +: +- Project +: +- BroadcastHashJoin [COMET: Comet is not compatible with Spark for case conversion in locale-specific cases. Set spark.comet.caseConversion.enabled=true to enable it anyway.] +: :- CometNativeColumnarToRow +: : +- CometProject +: : +- CometBroadcastHashJoin +: : :- CometProject +: : : +- CometBroadcastHashJoin +: : : :- CometProject +: : : : +- CometBroadcastHashJoin +: : : : :- CometProject +: : : : : +- CometSortMergeJoin +: : : : : :- CometSort +: : : : : : +- CometExchange +: : : : : : +- CometProject +: : : : : : +- CometFilter +: : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales +: : : : : +- CometSort +: : : : : +- CometExchange +: : : : : +- CometProject +: : : : : +- CometFilter +: : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns +: : : : +- CometBroadcastExchange +: : : : +- CometProject +: : : : +- CometFilter +: : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store +: : : +- CometBroadcastExchange +: : : +- CometProject +: : : +- CometFilter +: : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item +: : +- CometBroadcastExchange +: : +- CometProject +: : +- CometFilter +: : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer +: +- BroadcastExchange +: +- CometNativeColumnarToRow +: +- CometProject +: +- CometFilter +: +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address +- HashAggregate - +- CometNativeColumnarToRow - +- CometColumnarExchange + +- Exchange + +- HashAggregate +- HashAggregate - +- HashAggregate - +- CometNativeColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Project - +- BroadcastHashJoin [COMET: Comet is not compatible with Spark for case conversion in locale-specific cases. Set spark.comet.caseConversion.enabled=true to enable it anyway.] - :- CometNativeColumnarToRow - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometSortMergeJoin - : : : : :- CometSort - : : : : : +- CometExchange - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : +- CometSort - : : : : +- CometExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer - +- BroadcastExchange - +- CometNativeColumnarToRow - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address + +- Exchange + +- HashAggregate + +- Project + +- BroadcastHashJoin [COMET: Comet is not compatible with Spark for case conversion in locale-specific cases. Set spark.comet.caseConversion.enabled=true to enable it anyway.] + :- CometNativeColumnarToRow + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometSortMergeJoin + : : : : :- CometSort + : : : : : +- CometExchange + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales + : : : : +- CometSort + : : : : +- CometExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer + +- BroadcastExchange + +- CometNativeColumnarToRow + +- CometProject + +- CometFilter + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address -Comet accelerated 70 out of 86 eligible operators (81%). Final plan contains 8 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 66 out of 86 eligible operators (76%). 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/q24b.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q24b.native_datafusion/extended.txt index 03e6809902..8249b86a1e 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q24b.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q24b.native_datafusion/extended.txt @@ -1,96 +1,92 @@ Filter : +- Subquery : +- HashAggregate -: +- CometNativeColumnarToRow -: +- CometColumnarExchange +: +- Exchange +: +- HashAggregate : +- HashAggregate -: +- HashAggregate -: +- CometNativeColumnarToRow -: +- CometColumnarExchange -: +- HashAggregate -: +- Project -: +- BroadcastHashJoin [COMET: Comet is not compatible with Spark for case conversion in locale-specific cases. Set spark.comet.caseConversion.enabled=true to enable it anyway.] -: :- CometNativeColumnarToRow -: : +- CometProject -: : +- CometBroadcastHashJoin -: : :- CometProject -: : : +- CometBroadcastHashJoin -: : : :- CometProject -: : : : +- CometBroadcastHashJoin -: : : : :- CometProject -: : : : : +- CometSortMergeJoin -: : : : : :- CometSort -: : : : : : +- CometExchange -: : : : : : +- CometProject -: : : : : : +- CometFilter -: : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales -: : : : : +- CometSort -: : : : : +- CometExchange -: : : : : +- CometProject -: : : : : +- CometFilter -: : : : : +- CometNativeScan parquet spark_catalog.default.store_returns -: : : : +- CometBroadcastExchange -: : : : +- CometProject -: : : : +- CometFilter -: : : : +- CometNativeScan parquet spark_catalog.default.store -: : : +- CometBroadcastExchange -: : : +- CometProject -: : : +- CometFilter -: : : +- CometNativeScan parquet spark_catalog.default.item -: : +- CometBroadcastExchange -: : +- CometProject -: : +- CometFilter -: : +- CometNativeScan parquet spark_catalog.default.customer -: +- BroadcastExchange -: +- CometNativeColumnarToRow -: +- CometProject -: +- CometFilter -: +- CometNativeScan parquet spark_catalog.default.customer_address +: +- Exchange +: +- HashAggregate +: +- Project +: +- BroadcastHashJoin [COMET: Comet is not compatible with Spark for case conversion in locale-specific cases. Set spark.comet.caseConversion.enabled=true to enable it anyway.] +: :- CometNativeColumnarToRow +: : +- CometProject +: : +- CometBroadcastHashJoin +: : :- CometProject +: : : +- CometBroadcastHashJoin +: : : :- CometProject +: : : : +- CometBroadcastHashJoin +: : : : :- CometProject +: : : : : +- CometSortMergeJoin +: : : : : :- CometSort +: : : : : : +- CometExchange +: : : : : : +- CometProject +: : : : : : +- CometFilter +: : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales +: : : : : +- CometSort +: : : : : +- CometExchange +: : : : : +- CometProject +: : : : : +- CometFilter +: : : : : +- CometNativeScan parquet spark_catalog.default.store_returns +: : : : +- CometBroadcastExchange +: : : : +- CometProject +: : : : +- CometFilter +: : : : +- CometNativeScan parquet spark_catalog.default.store +: : : +- CometBroadcastExchange +: : : +- CometProject +: : : +- CometFilter +: : : +- CometNativeScan parquet spark_catalog.default.item +: : +- CometBroadcastExchange +: : +- CometProject +: : +- CometFilter +: : +- CometNativeScan parquet spark_catalog.default.customer +: +- BroadcastExchange +: +- CometNativeColumnarToRow +: +- CometProject +: +- CometFilter +: +- CometNativeScan parquet spark_catalog.default.customer_address +- HashAggregate - +- CometNativeColumnarToRow - +- CometColumnarExchange + +- Exchange + +- HashAggregate +- HashAggregate - +- HashAggregate - +- CometNativeColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Project - +- BroadcastHashJoin [COMET: Comet is not compatible with Spark for case conversion in locale-specific cases. Set spark.comet.caseConversion.enabled=true to enable it anyway.] - :- CometNativeColumnarToRow - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometSortMergeJoin - : : : : :- CometSort - : : : : : +- CometExchange - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : : +- CometSort - : : : : +- CometExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.store_returns - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.store - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.item - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.customer - +- BroadcastExchange - +- CometNativeColumnarToRow - +- CometProject - +- CometFilter - +- CometNativeScan parquet spark_catalog.default.customer_address + +- Exchange + +- HashAggregate + +- Project + +- BroadcastHashJoin [COMET: Comet is not compatible with Spark for case conversion in locale-specific cases. Set spark.comet.caseConversion.enabled=true to enable it anyway.] + :- CometNativeColumnarToRow + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometSortMergeJoin + : : : : :- CometSort + : : : : : +- CometExchange + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : +- CometSort + : : : : +- CometExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.store_returns + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.store + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.item + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.customer + +- BroadcastExchange + +- CometNativeColumnarToRow + +- CometProject + +- CometFilter + +- CometNativeScan parquet spark_catalog.default.customer_address -Comet accelerated 70 out of 86 eligible operators (81%). Final plan contains 8 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 66 out of 86 eligible operators (76%). 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/q24b.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q24b.native_iceberg_compat/extended.txt index 36748b40ea..fa983a36da 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q24b.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q24b.native_iceberg_compat/extended.txt @@ -1,96 +1,92 @@ Filter : +- Subquery : +- HashAggregate -: +- CometNativeColumnarToRow -: +- CometColumnarExchange +: +- Exchange +: +- HashAggregate : +- HashAggregate -: +- HashAggregate -: +- CometNativeColumnarToRow -: +- CometColumnarExchange -: +- HashAggregate -: +- Project -: +- BroadcastHashJoin [COMET: Comet is not compatible with Spark for case conversion in locale-specific cases. Set spark.comet.caseConversion.enabled=true to enable it anyway.] -: :- CometNativeColumnarToRow -: : +- CometProject -: : +- CometBroadcastHashJoin -: : :- CometProject -: : : +- CometBroadcastHashJoin -: : : :- CometProject -: : : : +- CometBroadcastHashJoin -: : : : :- CometProject -: : : : : +- CometSortMergeJoin -: : : : : :- CometSort -: : : : : : +- CometExchange -: : : : : : +- CometProject -: : : : : : +- CometFilter -: : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -: : : : : +- CometSort -: : : : : +- CometExchange -: : : : : +- CometProject -: : : : : +- CometFilter -: : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns -: : : : +- CometBroadcastExchange -: : : : +- CometProject -: : : : +- CometFilter -: : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store -: : : +- CometBroadcastExchange -: : : +- CometProject -: : : +- CometFilter -: : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item -: : +- CometBroadcastExchange -: : +- CometProject -: : +- CometFilter -: : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer -: +- BroadcastExchange -: +- CometNativeColumnarToRow -: +- CometProject -: +- CometFilter -: +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address +: +- Exchange +: +- HashAggregate +: +- Project +: +- BroadcastHashJoin [COMET: Comet is not compatible with Spark for case conversion in locale-specific cases. Set spark.comet.caseConversion.enabled=true to enable it anyway.] +: :- CometNativeColumnarToRow +: : +- CometProject +: : +- CometBroadcastHashJoin +: : :- CometProject +: : : +- CometBroadcastHashJoin +: : : :- CometProject +: : : : +- CometBroadcastHashJoin +: : : : :- CometProject +: : : : : +- CometSortMergeJoin +: : : : : :- CometSort +: : : : : : +- CometExchange +: : : : : : +- CometProject +: : : : : : +- CometFilter +: : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales +: : : : : +- CometSort +: : : : : +- CometExchange +: : : : : +- CometProject +: : : : : +- CometFilter +: : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns +: : : : +- CometBroadcastExchange +: : : : +- CometProject +: : : : +- CometFilter +: : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store +: : : +- CometBroadcastExchange +: : : +- CometProject +: : : +- CometFilter +: : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item +: : +- CometBroadcastExchange +: : +- CometProject +: : +- CometFilter +: : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer +: +- BroadcastExchange +: +- CometNativeColumnarToRow +: +- CometProject +: +- CometFilter +: +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address +- HashAggregate - +- CometNativeColumnarToRow - +- CometColumnarExchange + +- Exchange + +- HashAggregate +- HashAggregate - +- HashAggregate - +- CometNativeColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Project - +- BroadcastHashJoin [COMET: Comet is not compatible with Spark for case conversion in locale-specific cases. Set spark.comet.caseConversion.enabled=true to enable it anyway.] - :- CometNativeColumnarToRow - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometSortMergeJoin - : : : : :- CometSort - : : : : : +- CometExchange - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : +- CometSort - : : : : +- CometExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer - +- BroadcastExchange - +- CometNativeColumnarToRow - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address + +- Exchange + +- HashAggregate + +- Project + +- BroadcastHashJoin [COMET: Comet is not compatible with Spark for case conversion in locale-specific cases. Set spark.comet.caseConversion.enabled=true to enable it anyway.] + :- CometNativeColumnarToRow + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometSortMergeJoin + : : : : :- CometSort + : : : : : +- CometExchange + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales + : : : : +- CometSort + : : : : +- CometExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer + +- BroadcastExchange + +- CometNativeColumnarToRow + +- CometProject + +- CometFilter + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address -Comet accelerated 70 out of 86 eligible operators (81%). Final plan contains 8 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 66 out of 86 eligible operators (76%). 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/q25.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q25.native_datafusion/extended.txt index 15c6bec5dd..81047a7757 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,69 @@ 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 + +- Exchange + +- 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 -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 21 out of 57 eligible operators (36%). 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/q26.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q26.native_datafusion/extended.txt index ceba8f5828..9a298d216c 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,43 @@ 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 + +- Exchange + +- 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 -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 15 out of 35 eligible operators (42%). 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/q27.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q27.native_datafusion/extended.txt index 20f7517d3e..87d8460ec7 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,44 @@ 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 + +- Exchange + +- 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 -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 15 out of 36 eligible operators (41%). 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/q29.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q29.native_datafusion/extended.txt index bd9fa52aa7..98d6e2d24e 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,74 @@ 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 + +- Exchange + +- 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 -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 24 out of 61 eligible operators (39%). 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/q30.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q30.native_datafusion/extended.txt index c2e96eb412..659f71490a 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 @@ -7,60 +7,57 @@ TakeOrderedAndProject : : +- 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 + : : : +- Exchange + : : : +- 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 + : : +- Exchange + : : +- HashAggregate : : +- 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 + : : +- Exchange + : : +- 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 @@ -72,4 +69,4 @@ TakeOrderedAndProject +- 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 21 out of 61 eligible operators (34%). 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/q31.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q31.native_datafusion/extended.txt index 03bcf0413e..9b960f1ab1 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 @@ -10,141 +10,135 @@ CometNativeColumnarToRow : : : +- 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 + : : : : : +- Exchange + : : : : : +- 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 + : : : : +- Exchange + : : : : +- 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 + : : : +- Exchange + : : : +- 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 + : : +- Exchange + : : +- 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 + : +- Exchange + : +- 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 + +- Exchange + +- 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 -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 32 out of 120 eligible operators (26%). 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-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..69d5009252 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,46 @@ 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 ++- Exchange + +- 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 + : +- Exchange + : +- 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 -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 12 out of 38 eligible operators (31%). 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/q33.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q33.native_datafusion/extended.txt index 38b3efb123..b237976296 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,108 @@ 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 + +- Exchange + +- HashAggregate + +- Union + :- HashAggregate + : +- Exchange + : +- 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 + : +- Exchange + : +- 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 + +- Exchange + +- 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 -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 42 out of 93 eligible operators (45%). 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/q34.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q34.native_datafusion/extended.txt index 40230582bb..575c853e76 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 @@ -5,43 +5,42 @@ CometNativeColumnarToRow +- 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 + : +- Exchange + : +- 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 -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 17 out of 37 eligible operators (45%). 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/q35.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q35.native_datafusion/extended.txt index 40a8ef4a6e..c8a9740a02 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 @@ -1,67 +1,66 @@ TakeOrderedAndProject +- HashAggregate - +- CometNativeColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- 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 - : : : : +- 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 - : : : +- 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 - : : +- 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 + +- Exchange + +- HashAggregate + +- Project + +- BroadcastHashJoin + :- Project + : +- BroadcastHashJoin + : :- 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 + : : : : +- 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 + : : : +- 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 + : : +- 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 -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 20 out of 54 eligible operators (37%). 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/q35.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q35.native_iceberg_compat/extended.txt index c5125b1861..72960fc267 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q35.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q35.native_iceberg_compat/extended.txt @@ -1,63 +1,62 @@ TakeOrderedAndProject +- HashAggregate - +- CometNativeColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- Project - : : +- Filter - : : +- BroadcastHashJoin - : : :- BroadcastHashJoin [COMET: Unsupported join type ExistenceJoin(exists#1)] - : : : :- CometNativeColumnarToRow - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : +- ReusedSubquery - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : +- ReusedSubquery - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- CometNativeColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address - +- BroadcastExchange - +- CometNativeColumnarToRow - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics + +- Exchange + +- HashAggregate + +- Project + +- BroadcastHashJoin + :- Project + : +- BroadcastHashJoin + : :- Project + : : +- Filter + : : +- BroadcastHashJoin + : : :- BroadcastHashJoin [COMET: Unsupported join type ExistenceJoin(exists#1)] + : : : :- CometNativeColumnarToRow + : : : : +- CometBroadcastHashJoin + : : : : :- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometNativeColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : +- BroadcastExchange + : : : +- CometNativeColumnarToRow + : : : +- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales + : : : : +- ReusedSubquery + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- BroadcastExchange + : : +- CometNativeColumnarToRow + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales + : : : +- ReusedSubquery + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- BroadcastExchange + : +- CometNativeColumnarToRow + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address + +- BroadcastExchange + +- CometNativeColumnarToRow + +- CometProject + +- CometFilter + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics -Comet accelerated 35 out of 54 eligible operators (64%). Final plan contains 7 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 34 out of 54 eligible operators (62%). 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 aa40ed68cc..7b401d7eb8 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 @@ -5,39 +5,38 @@ TakeOrderedAndProject +- 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 + +- Exchange + +- 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 -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 14 out of 34 eligible operators (41%). 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/q39a.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q39a.native_datafusion/extended.txt index d9cec17b28..c38a41ba07 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 @@ -5,71 +5,69 @@ CometNativeColumnarToRow :- 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 + : +- Exchange + : +- 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 + +- Exchange + +- 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 -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 22 out of 60 eligible operators (36%). 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/q39b.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q39b.native_datafusion/extended.txt index d9cec17b28..c38a41ba07 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 @@ -5,71 +5,69 @@ CometNativeColumnarToRow :- 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 + : +- Exchange + : +- 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 + +- Exchange + +- 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 -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 22 out of 60 eligible operators (36%). 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/q4.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q4.native_datafusion/extended.txt index 737217ff91..d28d212f33 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 @@ -10,145 +10,139 @@ TakeOrderedAndProject : : : :- 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 + : : : : : +- Exchange + : : : : : +- 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 + : : : : +- Exchange + : : : : +- 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 + : : : +- Exchange + : : : +- 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 + : : +- Exchange + : : +- 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 + : +- Exchange + : +- 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 + +- Exchange + +- 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 -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 34 out of 126 eligible operators (26%). Final plan contains 20 transitions 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..4f2175cf7b 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 @@ -1,51 +1,50 @@ TakeOrderedAndProject +- HashAggregate - +- CometNativeColumnarToRow - +- CometColumnarExchange - +- 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 - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.item - +- BroadcastExchange - +- CometNativeColumnarToRow - +- CometProject - +- CometFilter - +- CometNativeScan parquet spark_catalog.default.item + +- Exchange + +- 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 + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.item + +- BroadcastExchange + +- CometNativeColumnarToRow + +- CometProject + +- 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 17 out of 41 eligible operators (41%). 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/q45.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q45.native_iceberg_compat/extended.txt index afdf8d6633..f8c94605ab 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q45.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q45.native_iceberg_compat/extended.txt @@ -1,47 +1,46 @@ TakeOrderedAndProject +- HashAggregate - +- CometNativeColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Project - +- Filter - +- BroadcastHashJoin [COMET: Unsupported join type ExistenceJoin(exists#1)] - :- CometNativeColumnarToRow - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - +- BroadcastExchange - +- CometNativeColumnarToRow - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + +- Exchange + +- HashAggregate + +- Project + +- Filter + +- BroadcastHashJoin [COMET: Unsupported join type ExistenceJoin(exists#1)] + :- CometNativeColumnarToRow + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometNativeColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + +- BroadcastExchange + +- CometNativeColumnarToRow + +- CometProject + +- CometFilter + +- CometScan [native_iceberg_compat] 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 31 out of 41 eligible operators (75%). 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 cebcd85144..7aaa58fcbf 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 @@ -4,45 +4,44 @@ TakeOrderedAndProject :- 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 + : : +- Exchange + : : +- 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 @@ -53,4 +52,4 @@ TakeOrderedAndProject +- 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 19 out of 45 eligible operators (42%). 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/q47.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q47.native_datafusion/extended.txt index ab44bbef5c..e347d0f433 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 @@ -12,36 +12,35 @@ TakeOrderedAndProject : : +- 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 + : : +- Exchange + : : +- 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 : +- 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).] @@ -49,36 +48,35 @@ TakeOrderedAndProject : +- 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 + : +- Exchange + : +- 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 +- 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).] @@ -86,35 +84,34 @@ TakeOrderedAndProject +- 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 + +- Exchange + +- 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 -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 33 out of 97 eligible operators (34%). 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-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..411a47f8ed 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,41 @@ 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 ++- Exchange + +- 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 -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 14 out of 33 eligible operators (42%). 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/q49.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q49.native_datafusion/extended.txt index 0090739d55..d19eb04caf 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 @@ -13,33 +13,32 @@ 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 + : +- Exchange + : +- 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 :- Project : +- Filter : +- Window @@ -49,28 +48,27 @@ 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 + : +- Exchange + : +- 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 +- Project +- Filter +- Window @@ -80,27 +78,26 @@ 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 + +- Exchange + +- 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 -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 30 out of 87 eligible operators (34%). 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-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..c9204f8e91 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,102 @@ 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 + +- Exchange + +- HashAggregate + +- Expand + +- Union + :- HashAggregate + : +- Exchange + : +- 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 + : +- Exchange + : +- 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 + +- Exchange + +- 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 -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 24 out of 86 eligible operators (27%). 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-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..d0b7b01cee 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,41 @@ 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 + +- Exchange + +- 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 -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 13 out of 33 eligible operators (39%). 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/q51.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q51.native_datafusion/extended.txt index db4e38a1bb..ee122e3dcd 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 @@ -14,25 +14,24 @@ TakeOrderedAndProject : +- 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 + : +- Exchange + : +- 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 +- CometSort +- CometColumnarExchange +- Project @@ -41,19 +40,18 @@ TakeOrderedAndProject +- 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 + +- Exchange + +- 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 -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 21 out of 47 eligible operators (44%). 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/q53.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q53.native_datafusion/extended.txt index 5c7a26057d..9b3651b3fb 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 @@ -6,37 +6,36 @@ TakeOrderedAndProject +- 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 + +- Exchange + +- 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 -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 13 out of 33 eligible operators (39%). 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/q54.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q54.native_datafusion/extended.txt index 273efea475..2d4a7b9185 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,114 @@ TakeOrderedAndProject +- HashAggregate - +- CometNativeColumnarToRow - +- CometColumnarExchange + +- Exchange + +- HashAggregate +- 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 + +- Exchange + +- 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 -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 49 out of 96 eligible operators (51%). Final plan contains 16 transitions 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..4babc2254d 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,111 @@ 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 + +- Exchange + +- HashAggregate + +- Union + :- HashAggregate + : +- Exchange + : +- 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 + : +- Exchange + : +- 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 + +- Exchange + +- 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 -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 45 out of 96 eligible operators (46%). 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/q57.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q57.native_datafusion/extended.txt index 66c510aaa0..50dc55f674 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 @@ -12,36 +12,35 @@ TakeOrderedAndProject : : +- 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 + : : +- Exchange + : : +- 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 : +- 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).] @@ -49,36 +48,35 @@ TakeOrderedAndProject : +- 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 + : +- Exchange + : +- 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 +- 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).] @@ -86,35 +84,34 @@ TakeOrderedAndProject +- 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 + +- Exchange + +- 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 -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 33 out of 97 eligible operators (34%). 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-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..f705219275 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 @@ -5,122 +5,119 @@ TakeOrderedAndProject : +- 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 + : : +- Exchange + : : +- 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 + : +- Exchange + : +- 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 + +- Exchange + +- 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 -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 55 out of 108 eligible operators (50%). 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/q6.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q6.native_datafusion/extended.txt index f2cdb50e03..18cc2a5c96 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,67 @@ 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 + +- Exchange + +- 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 -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 38 out of 58 eligible operators (65%). 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/q60.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q60.native_datafusion/extended.txt index ad055b48b6..4babc2254d 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,111 @@ 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 + +- Exchange + +- HashAggregate + +- Union + :- HashAggregate + : +- Exchange + : +- 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 + : +- Exchange + : +- 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 + +- Exchange + +- 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 -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 45 out of 96 eligible operators (46%). 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/q61.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q61.native_datafusion/extended.txt index 025280d4ca..da7b7fc2f0 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,99 @@ 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 + : +- Exchange + : +- 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 +- 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 + +- Exchange + +- 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 -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 34 out of 83 eligible operators (40%). 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-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..9b3651b3fb 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 @@ -6,37 +6,36 @@ TakeOrderedAndProject +- 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 + +- Exchange + +- 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 -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 13 out of 33 eligible operators (39%). 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/q65.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q65.native_datafusion/extended.txt index 73e3aa7046..f059866684 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 @@ -11,25 +11,24 @@ TakeOrderedAndProject : : +- 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 + : : +- Exchange + : : +- 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 @@ -38,23 +37,21 @@ TakeOrderedAndProject +- BroadcastExchange +- Filter +- HashAggregate - +- CometNativeColumnarToRow - +- CometColumnarExchange + +- Exchange + +- HashAggregate +- 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 + +- Exchange + +- 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 -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 14 out of 48 eligible operators (29%). 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/q66.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q66.native_datafusion/extended.txt index 3f3c6b8d79..b586928ede 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,79 @@ 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 + +- Exchange + +- HashAggregate + +- Union + :- HashAggregate + : +- Exchange + : +- 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 + +- Exchange + +- 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 -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 24 out of 66 eligible operators (36%). 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/q67.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q67.native_datafusion/extended.txt index 27b7a8ecb4..1ea44b0f90 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 @@ -5,39 +5,38 @@ TakeOrderedAndProject +- 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 + +- Exchange + +- 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 -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 14 out of 34 eligible operators (41%). 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/q68.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q68.native_datafusion/extended.txt index cebcd85144..7aaa58fcbf 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 @@ -4,45 +4,44 @@ TakeOrderedAndProject :- 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 + : : +- Exchange + : : +- 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 @@ -53,4 +52,4 @@ TakeOrderedAndProject +- 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 19 out of 45 eligible operators (42%). 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/q69.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q69.native_datafusion/extended.txt index f48c67d6e1..c8bb5a4bde 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 @@ -1,66 +1,65 @@ TakeOrderedAndProject +- HashAggregate - +- CometNativeColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Project - +- BroadcastHashJoin - :- Project - : +- 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 - : : : : +- 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 - : : : +- 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 - : : +- 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 + +- Exchange + +- HashAggregate + +- Project + +- BroadcastHashJoin + :- Project + : +- 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 + : : : : +- 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 + : : : +- 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 + : : +- 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 -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 20 out of 53 eligible operators (37%). 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/q69.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q69.native_iceberg_compat/extended.txt index bf624b5ce3..726fb6b283 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q69.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q69.native_iceberg_compat/extended.txt @@ -1,62 +1,61 @@ TakeOrderedAndProject +- HashAggregate - +- CometNativeColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- BroadcastHashJoin [COMET: BuildRight with LeftAnti is not supported] - : : : :- CometNativeColumnarToRow - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : +- ReusedSubquery - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : +- ReusedSubquery - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- CometNativeColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address - +- BroadcastExchange - +- CometNativeColumnarToRow - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics + +- Exchange + +- HashAggregate + +- Project + +- BroadcastHashJoin + :- Project + : +- BroadcastHashJoin + : :- Project + : : +- BroadcastHashJoin + : : :- BroadcastHashJoin [COMET: BuildRight with LeftAnti is not supported] + : : : :- CometNativeColumnarToRow + : : : : +- CometBroadcastHashJoin + : : : : :- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometNativeColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : +- BroadcastExchange + : : : +- CometNativeColumnarToRow + : : : +- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales + : : : : +- ReusedSubquery + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- BroadcastExchange + : : +- CometNativeColumnarToRow + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales + : : : +- ReusedSubquery + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- BroadcastExchange + : +- CometNativeColumnarToRow + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address + +- BroadcastExchange + +- CometNativeColumnarToRow + +- CometProject + +- CometFilter + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics -Comet accelerated 35 out of 53 eligible operators (66%). Final plan contains 7 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 34 out of 53 eligible operators (64%). 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 b692dfa9be..60c0417071 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,43 @@ 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 + +- Exchange + +- 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 -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 15 out of 35 eligible operators (42%). 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 546d2cc0bc..fcbf14b328 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 @@ -5,60 +5,58 @@ TakeOrderedAndProject +- CometSort +- CometColumnarExchange +- HashAggregate - +- CometNativeColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- 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 - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.date_dim - +- BroadcastExchange - +- Project - +- BroadcastHashJoin - :- CometNativeColumnarToRow - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.store - +- 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 + +- Exchange + +- HashAggregate + +- 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 + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.date_dim + +- BroadcastExchange + +- Project + +- BroadcastHashJoin + :- CometNativeColumnarToRow + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.store + +- BroadcastExchange + +- Project + +- Filter + +- Window + +- Sort + +- HashAggregate + +- Exchange + +- 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 -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 16 out of 52 eligible operators (30%). 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/q70.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q70.native_iceberg_compat/extended.txt index 14e4cb087a..730d37ec0a 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q70.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q70.native_iceberg_compat/extended.txt @@ -5,56 +5,55 @@ TakeOrderedAndProject +- CometSort +- CometColumnarExchange +- HashAggregate - +- CometNativeColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Expand - +- Project - +- BroadcastHashJoin - :- CometNativeColumnarToRow - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - +- BroadcastExchange - +- Project - +- BroadcastHashJoin - :- CometNativeColumnarToRow - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - +- BroadcastExchange - +- Project - +- Filter - +- 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 - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : +- ReusedSubquery - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + +- Exchange + +- HashAggregate + +- Expand + +- Project + +- BroadcastHashJoin + :- CometNativeColumnarToRow + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales + : : +- SubqueryBroadcast + : : +- BroadcastExchange + : : +- CometNativeColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + +- BroadcastExchange + +- Project + +- BroadcastHashJoin + :- CometNativeColumnarToRow + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store + +- BroadcastExchange + +- Project + +- Filter + +- 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 + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales + : : +- ReusedSubquery + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store + +- CometBroadcastExchange + +- CometProject + +- CometFilter + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Comet accelerated 34 out of 52 eligible operators (65%). Final plan contains 6 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 33 out of 52 eligible operators (63%). 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 23844f41f3..9d5665df4d 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 @@ -2,61 +2,60 @@ 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 + +- Exchange + +- 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 -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 20 out of 49 eligible operators (40%). 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/q73.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q73.native_datafusion/extended.txt index 40230582bb..575c853e76 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 @@ -5,43 +5,42 @@ CometNativeColumnarToRow +- 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 + : +- Exchange + : +- 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 -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 17 out of 37 eligible operators (45%). 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/q74.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q74.native_datafusion/extended.txt index 5d218998be..00d2b3b0d7 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 @@ -6,100 +6,96 @@ TakeOrderedAndProject : :- 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 + : : : +- Exchange + : : : +- 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 + : : +- Exchange + : : +- 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 + : +- Exchange + : +- 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 + +- Exchange + +- 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 -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 24 out of 85 eligible operators (28%). 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-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..e647d8db46 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 @@ -1,135 +1,128 @@ TakeOrderedAndProject +- HashAggregate - +- CometNativeColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Expand - +- Union - :- 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 - : :- 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 + +- Exchange + +- HashAggregate + +- Expand + +- Union + :- Project + : +- BroadcastHashJoin + : :- HashAggregate + : : +- Exchange + : : +- 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 + : +- Exchange + : +- 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 + : :- BroadcastExchange + : : +- HashAggregate + : : +- Exchange + : : +- 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 + : +- Exchange + : +- 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 + : +- Exchange + : +- 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 + +- Exchange + +- 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 -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 29 out of 109 eligible operators (26%). Final plan contains 17 transitions 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_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q77.native_iceberg_compat/extended.txt index 0cfc0a8417..b2653c7068 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q77.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q77.native_iceberg_compat/extended.txt @@ -1,117 +1,116 @@ TakeOrderedAndProject +- HashAggregate - +- CometNativeColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Expand - +- Union - :- CometNativeColumnarToRow - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometHashAggregate - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - : +- CometBroadcastExchange - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns - : : : +- ReusedSubquery - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - :- Project - : +- BroadcastNestedLoopJoin [COMET: BroadcastNestedLoopJoin is not supported] - : :- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometHashAggregate - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : +- ReusedSubquery - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometNativeColumnarToRow - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns - : : +- ReusedSubquery - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - +- CometNativeColumnarToRow - +- CometProject - +- CometBroadcastHashJoin - :- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : +- ReusedSubquery - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_page - +- CometBroadcastExchange - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns - : : +- ReusedSubquery - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - +- CometBroadcastExchange - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_page + +- Exchange + +- HashAggregate + +- Expand + +- Union + :- CometNativeColumnarToRow + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometHashAggregate + : : +- CometExchange + : : +- CometHashAggregate + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometNativeColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store + : +- CometBroadcastExchange + : +- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns + : : : +- ReusedSubquery + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store + :- Project + : +- BroadcastNestedLoopJoin [COMET: BroadcastNestedLoopJoin is not supported] + : :- BroadcastExchange + : : +- CometNativeColumnarToRow + : : +- CometHashAggregate + : : +- CometExchange + : : +- CometHashAggregate + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales + : : : +- ReusedSubquery + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- CometNativeColumnarToRow + : +- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns + : : +- ReusedSubquery + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + +- CometNativeColumnarToRow + +- CometProject + +- CometBroadcastHashJoin + :- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales + : : : +- ReusedSubquery + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_page + +- CometBroadcastExchange + +- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns + : : +- ReusedSubquery + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + +- CometBroadcastExchange + +- CometFilter + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_page -Comet accelerated 94 out of 109 eligible operators (86%). Final plan contains 6 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 93 out of 109 eligible operators (85%). 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/q79.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q79.native_datafusion/extended.txt index f24d730d8b..76bf23f16f 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 @@ -2,43 +2,42 @@ 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 + : +- Exchange + : +- 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 -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 15 out of 35 eligible operators (42%). 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/q8.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q8.native_datafusion/extended.txt index 8c9eb7b8c9..838592f73b 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,55 @@ 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 + +- Exchange + +- 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 -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 31 out of 48 eligible operators (64%). 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/q81.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q81.native_datafusion/extended.txt index 841b2ca52d..6133951418 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 @@ -7,60 +7,57 @@ TakeOrderedAndProject : : +- 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 + : : : +- Exchange + : : : +- 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 + : : +- Exchange + : : +- HashAggregate : : +- 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 + : : +- Exchange + : : +- 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 @@ -72,4 +69,4 @@ TakeOrderedAndProject +- 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 21 out of 61 eligible operators (34%). 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/q83.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q83.native_datafusion/extended.txt index eac3126bae..221f0dfe14 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 @@ -4,127 +4,124 @@ TakeOrderedAndProject :- 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 + : : +- Exchange + : : +- 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 + : +- Exchange + : +- 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 + +- Exchange + +- 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 -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 69 out of 114 eligible operators (60%). 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/q85.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q85.native_datafusion/extended.txt index aed7b365be..efab56234e 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,63 @@ 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 + +- Exchange + +- 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 -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 23 out of 52 eligible operators (44%). 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/q86.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q86.native_datafusion/extended.txt index a60a3158a4..7313f7e979 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 @@ -5,32 +5,31 @@ TakeOrderedAndProject +- 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 + +- Exchange + +- 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 -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 11 out of 28 eligible operators (39%). 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/q87.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q87.native_datafusion/extended.txt index 7a9e62d57c..772b0f4512 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 @@ -1,82 +1,81 @@ HashAggregate -+- CometNativeColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Project - +- BroadcastHashJoin - :- 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 - : +- 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 - +- 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 ++- Exchange + +- HashAggregate + +- Project + +- BroadcastHashJoin + :- 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 + : +- 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 + +- 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 -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 27 out of 66 eligible operators (40%). 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/q87.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q87.native_iceberg_compat/extended.txt index 5fddd74768..6218729c98 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q87.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q87.native_iceberg_compat/extended.txt @@ -1,73 +1,72 @@ HashAggregate -+- CometNativeColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Project - +- BroadcastHashJoin - :- BroadcastHashJoin [COMET: BuildRight with LeftAnti is not supported] - : :- CometNativeColumnarToRow - : : +- CometHashAggregate - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer - : +- BroadcastExchange - : +- CometNativeColumnarToRow - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : +- ReusedSubquery - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer - +- BroadcastExchange - +- CometNativeColumnarToRow - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : +- ReusedSubquery - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer ++- Exchange + +- HashAggregate + +- Project + +- BroadcastHashJoin + :- BroadcastHashJoin [COMET: BuildRight with LeftAnti is not supported] + : :- CometNativeColumnarToRow + : : +- CometHashAggregate + : : +- CometExchange + : : +- CometHashAggregate + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometNativeColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer + : +- BroadcastExchange + : +- CometNativeColumnarToRow + : +- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales + : : : +- ReusedSubquery + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer + +- BroadcastExchange + +- CometNativeColumnarToRow + +- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales + : : +- ReusedSubquery + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + +- CometBroadcastExchange + +- CometProject + +- CometFilter + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer -Comet accelerated 55 out of 66 eligible operators (83%). Final plan contains 5 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 54 out of 66 eligible operators (81%). 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 5c7a26057d..9b3651b3fb 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 @@ -6,37 +6,36 @@ TakeOrderedAndProject +- 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 + +- Exchange + +- 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 -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 13 out of 33 eligible operators (39%). 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/q91.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q91.native_datafusion/extended.txt index 6f982615ae..27bfd61c53 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 @@ -2,58 +2,57 @@ 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 + +- Exchange + +- 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 -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 22 out of 47 eligible operators (46%). 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/q92.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q92.native_datafusion/extended.txt index b3ca85cdaf..c0f40501d8 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,46 @@ 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 ++- Exchange + +- 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 + : +- Exchange + : +- 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 -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 12 out of 38 eligible operators (31%). 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/q98.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q98.native_datafusion/extended.txt index af8cf29205..9de7969459 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 @@ -8,31 +8,30 @@ 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 + +- Exchange + +- 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 -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 14 out of 29 eligible operators (48%). 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/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..07ef74ce3d 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,64 @@ 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 + +- Exchange + +- 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 -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 20 out of 52 eligible operators (38%). 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/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..00d2b3b0d7 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 @@ -6,100 +6,96 @@ TakeOrderedAndProject : :- 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 + : : : +- Exchange + : : : +- 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 + : : +- Exchange + : : +- 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 + : +- Exchange + : +- 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 + +- Exchange + +- 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 -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 24 out of 85 eligible operators (28%). 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/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..79e4955126 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 @@ -5,31 +5,30 @@ TakeOrderedAndProject +- 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 + +- Exchange + +- 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 -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 11 out of 27 eligible operators (40%). 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/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..8d34a30261 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 @@ -3,402 +3,399 @@ TakeOrderedAndProject :- 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 + : : +- Exchange + : : +- 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 + : +- Exchange + : +- 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 + +- Exchange + +- 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 -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 125 out of 333 eligible operators (37%). Final plan contains 66 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..0245ef1d1c 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 @@ -5,2775 +5,2746 @@ CometNativeColumnarToRow +- 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 - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- Exchange + : +- HashAggregate + : +- Union + : :- Filter + : : : +- Subquery + : : : +- HashAggregate + : : : +- Exchange + : : : +- 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 + : : +- Exchange + : : +- 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 + : : +- Exchange + : : +- 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 + : +- Exchange + : +- 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 + : +- Exchange + : +- HashAggregate : +- 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 + : +- Exchange + : +- HashAggregate + : +- Union + : :- Filter + : : : +- Subquery + : : : +- HashAggregate + : : : +- Exchange + : : : +- 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 + : : +- Exchange + : : +- 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 + : : +- Exchange + : : +- 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 + : +- Exchange : +- 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 + : +- 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 + : +- Exchange + : +- HashAggregate : +- 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 + : +- Exchange + : +- HashAggregate + : +- Union + : :- Filter + : : : +- Subquery + : : : +- HashAggregate + : : : +- Exchange + : : : +- 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 + : : +- Exchange + : : +- 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 + : : +- Exchange + : : +- 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 + : +- Exchange : +- 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 + : +- 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 + : +- Exchange + : +- HashAggregate : +- 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 + : +- Exchange + : +- HashAggregate + : +- Union + : :- Filter + : : : +- Subquery + : : : +- HashAggregate + : : : +- Exchange + : : : +- 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 + : : +- Exchange + : : +- 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 + : : +- Exchange + : : +- 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 + : +- Exchange : +- 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 + : +- 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 + +- Exchange + +- HashAggregate +- 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 + +- Exchange + +- HashAggregate + +- Union + :- Filter + : : +- Subquery + : : +- HashAggregate + : : +- Exchange + : : +- 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 + : +- Exchange + : +- 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 + : +- Exchange + : +- 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 + +- Exchange +- 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 + +- 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 -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 813 out of 2302 eligible operators (35%). Final plan contains 446 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..1a1a476899 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,248 @@ 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 + : +- Exchange + : +- 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 + : +- Exchange + : +- 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 + : +- Exchange + : +- 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 + : +- Exchange + : +- 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 + +- Exchange + +- 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 -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 87 out of 210 eligible operators (41%). 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/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..74b85d1536 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 @@ -5,31 +5,30 @@ TakeOrderedAndProject +- 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 + +- Exchange + +- 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 -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 11 out of 27 eligible operators (40%). 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/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..bf53cec292 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 @@ -1,36 +1,35 @@ TakeOrderedAndProject +- HashAggregate - +- CometNativeColumnarToRow - +- CometColumnarExchange - +- 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 - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.item - +- BroadcastExchange - +- CometNativeColumnarToRow - +- CometNativeScan parquet spark_catalog.default.warehouse + +- Exchange + +- 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 + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.item + +- BroadcastExchange + +- 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 10 out of 28 eligible operators (35%). 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/q22.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q22.native_iceberg_compat/extended.txt index f642ebc712..136315f3db 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q22.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q22.native_iceberg_compat/extended.txt @@ -1,34 +1,33 @@ TakeOrderedAndProject +- HashAggregate - +- CometNativeColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Expand - +- Project - +- BroadcastNestedLoopJoin [COMET: BroadcastNestedLoopJoin is not supported] - :- CometNativeColumnarToRow - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.inventory - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - +- BroadcastExchange - +- CometNativeColumnarToRow - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.warehouse + +- Exchange + +- HashAggregate + +- Expand + +- Project + +- BroadcastNestedLoopJoin [COMET: BroadcastNestedLoopJoin is not supported] + :- CometNativeColumnarToRow + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.inventory + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometNativeColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + +- BroadcastExchange + +- CometNativeColumnarToRow + +- CometScan [native_iceberg_compat] 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 18 out of 28 eligible operators (64%). 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 eaa5438d91..508e7619dc 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 @@ -3,8 +3,42 @@ TakeOrderedAndProject :- HashAggregate : +- HashAggregate : +- HashAggregate - : +- CometNativeColumnarToRow - : +- CometColumnarExchange + : +- Exchange + : +- 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 + : +- Exchange + : +- HashAggregate + : +- HashAggregate + : +- Exchange : +- HashAggregate : +- Project : +- BroadcastHashJoin @@ -36,152 +70,109 @@ TakeOrderedAndProject : +- 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 + : +- Exchange + : +- 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 + : +- Exchange + : +- 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 + : +- Exchange + : +- 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 + : +- Exchange + : +- 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 + +- Exchange + +- 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 + +- Exchange + +- 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 -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 55 out of 151 eligible operators (36%). 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-spark3_5/q24.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q24.native_datafusion/extended.txt index 4c22965f0f..8e67cf3e18 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q24.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q24.native_datafusion/extended.txt @@ -4,96 +4,92 @@ CometNativeColumnarToRow +- Filter : +- Subquery : +- HashAggregate - : +- CometNativeColumnarToRow - : +- CometColumnarExchange + : +- Exchange + : +- HashAggregate : +- HashAggregate - : +- HashAggregate - : +- CometNativeColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin [COMET: Comet is not compatible with Spark for case conversion in locale-specific cases. Set spark.comet.caseConversion.enabled=true to enable it anyway.] - : :- CometNativeColumnarToRow - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometSortMergeJoin - : : : : : :- CometSort - : : : : : : +- CometExchange - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : : : +- CometSort - : : : : : +- CometExchange - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.store_returns - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.store - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.item - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.customer - : +- BroadcastExchange - : +- CometNativeColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.customer_address + : +- Exchange + : +- HashAggregate + : +- Project + : +- BroadcastHashJoin [COMET: Comet is not compatible with Spark for case conversion in locale-specific cases. Set spark.comet.caseConversion.enabled=true to enable it anyway.] + : :- CometNativeColumnarToRow + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometProject + : : : : : +- CometSortMergeJoin + : : : : : :- CometSort + : : : : : : +- CometExchange + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : : +- CometSort + : : : : : +- CometExchange + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.store_returns + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.store + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.item + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.customer + : +- BroadcastExchange + : +- CometNativeColumnarToRow + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.customer_address +- HashAggregate - +- CometNativeColumnarToRow - +- CometColumnarExchange + +- Exchange + +- HashAggregate +- HashAggregate - +- HashAggregate - +- CometNativeColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Project - +- BroadcastHashJoin [COMET: Comet is not compatible with Spark for case conversion in locale-specific cases. Set spark.comet.caseConversion.enabled=true to enable it anyway.] - :- CometNativeColumnarToRow - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometSortMergeJoin - : : : : :- CometSort - : : : : : +- CometExchange - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : : +- CometSort - : : : : +- CometExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.store_returns - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.store - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.item - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.customer - +- BroadcastExchange - +- CometNativeColumnarToRow - +- CometProject - +- CometFilter - +- CometNativeScan parquet spark_catalog.default.customer_address + +- Exchange + +- HashAggregate + +- Project + +- BroadcastHashJoin [COMET: Comet is not compatible with Spark for case conversion in locale-specific cases. Set spark.comet.caseConversion.enabled=true to enable it anyway.] + :- CometNativeColumnarToRow + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometSortMergeJoin + : : : : :- CometSort + : : : : : +- CometExchange + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : +- CometSort + : : : : +- CometExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.store_returns + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.store + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.item + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.customer + +- BroadcastExchange + +- CometNativeColumnarToRow + +- CometProject + +- CometFilter + +- CometNativeScan parquet spark_catalog.default.customer_address -Comet accelerated 72 out of 88 eligible operators (81%). Final plan contains 9 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 68 out of 88 eligible operators (77%). 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/q24.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q24.native_iceberg_compat/extended.txt index 5d8bc6a064..4488b9b0ff 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q24.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q24.native_iceberg_compat/extended.txt @@ -4,96 +4,92 @@ CometNativeColumnarToRow +- Filter : +- Subquery : +- HashAggregate - : +- CometNativeColumnarToRow - : +- CometColumnarExchange + : +- Exchange + : +- HashAggregate : +- HashAggregate - : +- HashAggregate - : +- CometNativeColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin [COMET: Comet is not compatible with Spark for case conversion in locale-specific cases. Set spark.comet.caseConversion.enabled=true to enable it anyway.] - : :- CometNativeColumnarToRow - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometSortMergeJoin - : : : : : :- CometSort - : : : : : : +- CometExchange - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : +- CometSort - : : : : : +- CometExchange - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer - : +- BroadcastExchange - : +- CometNativeColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address + : +- Exchange + : +- HashAggregate + : +- Project + : +- BroadcastHashJoin [COMET: Comet is not compatible with Spark for case conversion in locale-specific cases. Set spark.comet.caseConversion.enabled=true to enable it anyway.] + : :- CometNativeColumnarToRow + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometProject + : : : : : +- CometSortMergeJoin + : : : : : :- CometSort + : : : : : : +- CometExchange + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales + : : : : : +- CometSort + : : : : : +- CometExchange + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer + : +- BroadcastExchange + : +- CometNativeColumnarToRow + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address +- HashAggregate - +- CometNativeColumnarToRow - +- CometColumnarExchange + +- Exchange + +- HashAggregate +- HashAggregate - +- HashAggregate - +- CometNativeColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Project - +- BroadcastHashJoin [COMET: Comet is not compatible with Spark for case conversion in locale-specific cases. Set spark.comet.caseConversion.enabled=true to enable it anyway.] - :- CometNativeColumnarToRow - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometSortMergeJoin - : : : : :- CometSort - : : : : : +- CometExchange - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : +- CometSort - : : : : +- CometExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer - +- BroadcastExchange - +- CometNativeColumnarToRow - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address + +- Exchange + +- HashAggregate + +- Project + +- BroadcastHashJoin [COMET: Comet is not compatible with Spark for case conversion in locale-specific cases. Set spark.comet.caseConversion.enabled=true to enable it anyway.] + :- CometNativeColumnarToRow + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometSortMergeJoin + : : : : :- CometSort + : : : : : +- CometExchange + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales + : : : : +- CometSort + : : : : +- CometExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer + +- BroadcastExchange + +- CometNativeColumnarToRow + +- CometProject + +- CometFilter + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address -Comet accelerated 72 out of 88 eligible operators (81%). Final plan contains 9 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 68 out of 88 eligible operators (77%). 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/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..793de3abde 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,113 @@ 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 + : +- Exchange + : +- 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 + : +- Exchange + : +- 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 + +- Exchange + +- 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 -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 38 out of 95 eligible operators (40%). Final plan contains 16 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..575c853e76 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 @@ -5,43 +5,42 @@ CometNativeColumnarToRow +- 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 + : +- Exchange + : +- 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 -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 17 out of 37 eligible operators (45%). 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/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..c8a9740a02 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 @@ -1,67 +1,66 @@ TakeOrderedAndProject +- HashAggregate - +- CometNativeColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- 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 - : : : : +- 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 - : : : +- 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 - : : +- 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 + +- Exchange + +- HashAggregate + +- Project + +- BroadcastHashJoin + :- Project + : +- BroadcastHashJoin + : :- 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 + : : : : +- 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 + : : : +- 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 + : : +- 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 -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 20 out of 54 eligible operators (37%). 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/q35.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q35.native_iceberg_compat/extended.txt index c5125b1861..72960fc267 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q35.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q35.native_iceberg_compat/extended.txt @@ -1,63 +1,62 @@ TakeOrderedAndProject +- HashAggregate - +- CometNativeColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- Project - : : +- Filter - : : +- BroadcastHashJoin - : : :- BroadcastHashJoin [COMET: Unsupported join type ExistenceJoin(exists#1)] - : : : :- CometNativeColumnarToRow - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : +- ReusedSubquery - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : +- ReusedSubquery - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- CometNativeColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address - +- BroadcastExchange - +- CometNativeColumnarToRow - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics + +- Exchange + +- HashAggregate + +- Project + +- BroadcastHashJoin + :- Project + : +- BroadcastHashJoin + : :- Project + : : +- Filter + : : +- BroadcastHashJoin + : : :- BroadcastHashJoin [COMET: Unsupported join type ExistenceJoin(exists#1)] + : : : :- CometNativeColumnarToRow + : : : : +- CometBroadcastHashJoin + : : : : :- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometNativeColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : +- BroadcastExchange + : : : +- CometNativeColumnarToRow + : : : +- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales + : : : : +- ReusedSubquery + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- BroadcastExchange + : : +- CometNativeColumnarToRow + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales + : : : +- ReusedSubquery + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- BroadcastExchange + : +- CometNativeColumnarToRow + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address + +- BroadcastExchange + +- CometNativeColumnarToRow + +- CometProject + +- CometFilter + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics -Comet accelerated 35 out of 54 eligible operators (64%). Final plan contains 7 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 34 out of 54 eligible operators (62%). 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 9091536e96..07ef74ce3d 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,64 @@ 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 + +- Exchange + +- 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 -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 20 out of 52 eligible operators (38%). 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/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..380076587c 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 @@ -9,114 +9,109 @@ TakeOrderedAndProject +- 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 + : +- Exchange + : +- 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 + : +- Exchange + : +- HashAggregate : +- 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 + : +- Exchange + : +- 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 + +- Exchange + +- HashAggregate +- 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 + +- Exchange + +- 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 -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 40 out of 99 eligible operators (40%). Final plan contains 16 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..e347d0f433 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 @@ -12,36 +12,35 @@ TakeOrderedAndProject : : +- 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 + : : +- Exchange + : : +- 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 : +- 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).] @@ -49,36 +48,35 @@ TakeOrderedAndProject : +- 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 + : +- Exchange + : +- 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 +- 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).] @@ -86,35 +84,34 @@ TakeOrderedAndProject +- 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 + +- Exchange + +- 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 -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 33 out of 97 eligible operators (34%). 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/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..d19eb04caf 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 @@ -13,33 +13,32 @@ 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 + : +- Exchange + : +- 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 :- Project : +- Filter : +- Window @@ -49,28 +48,27 @@ 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 + : +- Exchange + : +- 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 +- Project +- Filter +- Window @@ -80,27 +78,26 @@ 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 + +- Exchange + +- 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 -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 30 out of 87 eligible operators (34%). 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/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..7bccd55531 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 @@ -14,111 +14,105 @@ TakeOrderedAndProject : :- CometSort : : +- CometColumnarExchange : : +- HashAggregate - : : +- CometNativeColumnarToRow - : : +- CometColumnarExchange - : : +- HashAggregate - : : +- Project - : : +- BroadcastHashJoin - : : :- 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 - : : +- 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 + : : +- Exchange + : : +- HashAggregate + : : +- Project + : : +- BroadcastHashJoin + : : :- 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 + : : : +- Exchange + : : : +- 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 + : : +- 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 + : : +- Exchange + : : +- 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 : +- CometSort : +- CometColumnarExchange : +- HashAggregate - : +- CometNativeColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- 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 - : +- 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 + : +- Exchange + : +- HashAggregate + : +- Project + : +- BroadcastHashJoin + : :- 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 + : : +- Exchange + : : +- 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 + : +- 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 + : +- Exchange + : +- 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 +- 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).] @@ -131,110 +125,104 @@ TakeOrderedAndProject :- CometSort : +- CometColumnarExchange : +- HashAggregate - : +- CometNativeColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- 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 - : +- 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 + : +- Exchange + : +- HashAggregate + : +- Project + : +- BroadcastHashJoin + : :- 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 + : : +- Exchange + : : +- 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 + : +- 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 + : +- Exchange + : +- 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 +- CometSort +- CometColumnarExchange +- HashAggregate - +- CometNativeColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Project - +- BroadcastHashJoin - :- 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 - +- 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 + +- Exchange + +- HashAggregate + +- Project + +- BroadcastHashJoin + :- 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 + : +- Exchange + : +- 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 + +- 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 + +- Exchange + +- 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 -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 70 out of 196 eligible operators (35%). Final plan contains 30 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_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q51a.native_iceberg_compat/extended.txt index 6560795de1..190830204f 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q51a.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q51a.native_iceberg_compat/extended.txt @@ -14,99 +14,97 @@ TakeOrderedAndProject : :- CometSort : : +- CometColumnarExchange : : +- HashAggregate - : : +- CometNativeColumnarToRow - : : +- CometColumnarExchange - : : +- HashAggregate - : : +- Project - : : +- BroadcastHashJoin - : : :- 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 - : : : +- CometExchange - : : : +- CometHashAggregate - : : : +- CometExchange - : : : +- CometHashAggregate - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] 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 - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- Exchange + : : +- HashAggregate + : : +- Project + : : +- BroadcastHashJoin + : : :- 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 + : : : +- CometExchange + : : : +- CometHashAggregate + : : : +- CometExchange + : : : +- CometHashAggregate + : : : +- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometNativeColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] 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 + : : +- CometExchange + : : +- CometHashAggregate + : : +- CometExchange + : : +- CometHashAggregate + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometNativeColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim : +- CometSort : +- CometColumnarExchange : +- HashAggregate - : +- CometNativeColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- 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 - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : +- ReusedSubquery - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] 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 - : +- CometExchange - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : +- ReusedSubquery - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- Exchange + : +- HashAggregate + : +- Project + : +- BroadcastHashJoin + : :- 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 + : : +- CometExchange + : : +- CometHashAggregate + : : +- CometExchange + : : +- CometHashAggregate + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales + : : : +- ReusedSubquery + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] 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 + : +- CometExchange + : +- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales + : : +- ReusedSubquery + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] 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).] @@ -119,98 +117,96 @@ TakeOrderedAndProject :- CometSort : +- CometColumnarExchange : +- HashAggregate - : +- CometNativeColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- 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 - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] 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 - : +- CometExchange - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- Exchange + : +- HashAggregate + : +- Project + : +- BroadcastHashJoin + : :- 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 + : : +- CometExchange + : : +- CometHashAggregate + : : +- CometExchange + : : +- CometHashAggregate + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometNativeColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] 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 + : +- CometExchange + : +- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales + : : +- SubqueryBroadcast + : : +- BroadcastExchange + : : +- CometNativeColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim +- CometSort +- CometColumnarExchange +- HashAggregate - +- CometNativeColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Project - +- BroadcastHashJoin - :- 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 - : +- CometExchange - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : +- ReusedSubquery - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] 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 - +- CometExchange - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : +- ReusedSubquery - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + +- Exchange + +- HashAggregate + +- Project + +- BroadcastHashJoin + :- 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 + : +- CometExchange + : +- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales + : : +- ReusedSubquery + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] 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 + +- CometExchange + +- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometProject + +- CometBroadcastHashJoin + :- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales + : +- ReusedSubquery + +- CometBroadcastExchange + +- CometProject + +- CometFilter + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Comet accelerated 138 out of 196 eligible operators (70%). Final plan contains 18 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 134 out of 196 eligible operators (68%). 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 66c510aaa0..50dc55f674 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 @@ -12,36 +12,35 @@ TakeOrderedAndProject : : +- 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 + : : +- Exchange + : : +- 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 : +- 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).] @@ -49,36 +48,35 @@ TakeOrderedAndProject : +- 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 + : +- Exchange + : +- 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 +- 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).] @@ -86,35 +84,34 @@ TakeOrderedAndProject +- 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 + +- Exchange + +- 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 -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 33 out of 97 eligible operators (34%). 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/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..858a24872a 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 @@ -5,318 +5,304 @@ CometNativeColumnarToRow +- 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 - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.web_site + : +- Exchange + : +- HashAggregate + : +- Union + : :- HashAggregate + : : +- Exchange + : : +- 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 + : : +- Exchange + : : +- 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 + : +- Exchange + : +- 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 + : +- Exchange + : +- HashAggregate : +- HashAggregate - : +- HashAggregate - : +- CometNativeColumnarToRow - : +- CometColumnarExchange + : +- Exchange + : +- HashAggregate + : +- Union + : :- HashAggregate + : : +- Exchange + : : +- 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 + : : +- Exchange + : : +- 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 - : +- 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 + : +- Exchange : +- 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 + : +- 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 + +- Exchange + +- HashAggregate +- HashAggregate - +- HashAggregate - +- CometNativeColumnarToRow - +- CometColumnarExchange + +- Exchange + +- HashAggregate + +- Union + :- HashAggregate + : +- Exchange + : +- 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 + : +- Exchange + : +- 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 - +- 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 + +- Exchange +- 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 + +- 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 -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 75 out of 263 eligible operators (28%). Final plan contains 43 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..18cc2a5c96 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,67 @@ 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 + +- Exchange + +- 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 -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 38 out of 58 eligible operators (65%). 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/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..245dc7a4d5 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 @@ -9,342 +9,325 @@ TakeOrderedAndProject +- 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 + : +- Exchange + : +- 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 + : +- Exchange + : +- HashAggregate : +- 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 + : +- Exchange + : +- 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 + : +- Exchange + : +- HashAggregate : +- 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 + : +- Exchange + : +- 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 + : +- Exchange + : +- HashAggregate : +- 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 + : +- Exchange + : +- 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 + : +- Exchange + : +- HashAggregate : +- 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 + : +- Exchange + : +- 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 + : +- Exchange + : +- HashAggregate : +- 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 + : +- Exchange + : +- 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 + : +- Exchange + : +- HashAggregate : +- 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 + : +- Exchange + : +- 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 + : +- Exchange + : +- HashAggregate : +- 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 + : +- Exchange + : +- 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 + +- Exchange + +- HashAggregate +- 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 + +- Exchange + +- 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 -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 110 out of 285 eligible operators (38%). Final plan contains 46 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..70155e92ca 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 @@ -9,180 +9,172 @@ TakeOrderedAndProject +- HashAggregate +- Union :- 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 - : +- Project - : +- BroadcastHashJoin - : :- CometNativeColumnarToRow - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.store - : +- BroadcastExchange - : +- 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 + : +- Exchange + : +- 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 + : +- Project + : +- BroadcastHashJoin + : :- CometNativeColumnarToRow + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.store + : +- BroadcastExchange + : +- Project + : +- Filter + : +- Window + : +- WindowGroupLimit + : +- Sort + : +- HashAggregate + : +- Exchange + : +- 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 :- HashAggregate - : +- CometNativeColumnarToRow - : +- CometColumnarExchange + : +- Exchange + : +- HashAggregate : +- 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 - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- Project - : +- BroadcastHashJoin - : :- CometNativeColumnarToRow - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.store - : +- BroadcastExchange - : +- Project - : +- Filter - : +- Window - : +- WindowGroupLimit - : +- Sort + : +- Exchange + : +- 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 + : +- Project + : +- BroadcastHashJoin + : :- CometNativeColumnarToRow + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.store + : +- BroadcastExchange + : +- Project + : +- Filter + : +- Window + : +- WindowGroupLimit + : +- Sort + : +- HashAggregate + : +- Exchange : +- 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 + : +- 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 +- HashAggregate - +- CometNativeColumnarToRow - +- CometColumnarExchange + +- Exchange + +- HashAggregate +- 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 - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.date_dim - +- BroadcastExchange - +- Project - +- BroadcastHashJoin - :- CometNativeColumnarToRow - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.store - +- BroadcastExchange - +- Project - +- Filter - +- Window - +- WindowGroupLimit - +- Sort + +- Exchange + +- 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 + +- Project + +- BroadcastHashJoin + :- CometNativeColumnarToRow + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.store + +- BroadcastExchange + +- Project + +- Filter + +- Window + +- WindowGroupLimit + +- Sort + +- HashAggregate + +- Exchange +- 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 + +- 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 -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 46 out of 156 eligible operators (29%). 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/q70a.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q70a.native_iceberg_compat/extended.txt index b21df154e6..ff6d42786b 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q70a.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q70a.native_iceberg_compat/extended.txt @@ -9,168 +9,163 @@ TakeOrderedAndProject +- HashAggregate +- Union :- HashAggregate - : +- CometNativeColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- CometNativeColumnarToRow - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- Project - : +- BroadcastHashJoin - : :- CometNativeColumnarToRow - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - : +- BroadcastExchange - : +- Project - : +- Filter - : +- Window - : +- WindowGroupLimit [COMET: WindowGroupLimit is not supported] - : +- CometNativeColumnarToRow - : +- CometSort - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : +- ReusedSubquery - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- Exchange + : +- HashAggregate + : +- Project + : +- BroadcastHashJoin + : :- CometNativeColumnarToRow + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometNativeColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- BroadcastExchange + : +- Project + : +- BroadcastHashJoin + : :- CometNativeColumnarToRow + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store + : +- BroadcastExchange + : +- Project + : +- Filter + : +- Window + : +- WindowGroupLimit [COMET: WindowGroupLimit is not supported] + : +- CometNativeColumnarToRow + : +- CometSort + : +- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales + : : : +- ReusedSubquery + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim :- HashAggregate - : +- CometNativeColumnarToRow - : +- CometColumnarExchange + : +- Exchange + : +- HashAggregate : +- HashAggregate - : +- HashAggregate - : +- CometNativeColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- CometNativeColumnarToRow - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- Project - : +- BroadcastHashJoin - : :- CometNativeColumnarToRow - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - : +- BroadcastExchange - : +- Project - : +- Filter - : +- Window - : +- WindowGroupLimit [COMET: WindowGroupLimit is not supported] - : +- CometNativeColumnarToRow - : +- CometSort + : +- Exchange + : +- HashAggregate + : +- Project + : +- BroadcastHashJoin + : :- CometNativeColumnarToRow + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometNativeColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- BroadcastExchange + : +- Project + : +- BroadcastHashJoin + : :- CometNativeColumnarToRow + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store + : +- BroadcastExchange + : +- Project + : +- Filter + : +- Window + : +- WindowGroupLimit [COMET: WindowGroupLimit is not supported] + : +- CometNativeColumnarToRow + : +- CometSort + : +- CometHashAggregate + : +- CometExchange : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : +- ReusedSubquery - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales + : : : +- ReusedSubquery + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim +- HashAggregate - +- CometNativeColumnarToRow - +- CometColumnarExchange + +- Exchange + +- HashAggregate +- HashAggregate - +- HashAggregate - +- CometNativeColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Project - +- BroadcastHashJoin - :- CometNativeColumnarToRow - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - +- BroadcastExchange - +- Project - +- BroadcastHashJoin - :- CometNativeColumnarToRow - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - +- BroadcastExchange - +- Project - +- Filter - +- Window - +- WindowGroupLimit [COMET: WindowGroupLimit is not supported] - +- CometNativeColumnarToRow - +- CometSort + +- Exchange + +- HashAggregate + +- Project + +- BroadcastHashJoin + :- CometNativeColumnarToRow + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales + : : +- SubqueryBroadcast + : : +- BroadcastExchange + : : +- CometNativeColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + +- BroadcastExchange + +- Project + +- BroadcastHashJoin + :- CometNativeColumnarToRow + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store + +- BroadcastExchange + +- Project + +- Filter + +- Window + +- WindowGroupLimit [COMET: WindowGroupLimit is not supported] + +- CometNativeColumnarToRow + +- CometSort + +- CometHashAggregate + +- CometExchange +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : +- ReusedSubquery - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales + : : +- ReusedSubquery + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store + +- CometBroadcastExchange + +- CometProject + +- CometFilter + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Comet accelerated 102 out of 156 eligible operators (65%). Final plan contains 18 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 97 out of 156 eligible operators (62%). 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-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..00d2b3b0d7 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 @@ -6,100 +6,96 @@ TakeOrderedAndProject : :- 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 + : : : +- Exchange + : : : +- 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 + : : +- Exchange + : : +- 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 + : +- Exchange + : +- 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 + +- Exchange + +- 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 -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 24 out of 85 eligible operators (28%). 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/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..d11e87a30b 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 @@ -5,405 +5,382 @@ CometNativeColumnarToRow +- HashAggregate +- Union :- HashAggregate - : +- CometNativeColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Union - : :- 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 - : : :- 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 + : +- Exchange + : +- HashAggregate + : +- Union + : :- Project + : : +- BroadcastHashJoin + : : :- HashAggregate + : : : +- Exchange + : : : +- 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 + : : +- Exchange + : : +- 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 + : : :- BroadcastExchange + : : : +- HashAggregate + : : : +- Exchange + : : : +- 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 + : : +- Exchange + : : +- 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 + : : +- Exchange + : : +- 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 + : +- Exchange + : +- 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 :- HashAggregate - : +- CometNativeColumnarToRow - : +- CometColumnarExchange + : +- Exchange + : +- HashAggregate : +- HashAggregate - : +- HashAggregate - : +- CometNativeColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Union - : :- 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 - : : :- 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 + : +- Exchange + : +- HashAggregate + : +- Union + : :- Project + : : +- BroadcastHashJoin + : : :- HashAggregate + : : : +- Exchange + : : : +- 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 + : : +- Exchange + : : +- 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 + : : :- BroadcastExchange + : : : +- HashAggregate + : : : +- Exchange + : : : +- 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 + : : +- Exchange + : : +- 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 + : : +- Exchange : : +- 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 + : : +- 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 + : +- Exchange : +- 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 + : +- 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 +- HashAggregate - +- CometNativeColumnarToRow - +- CometColumnarExchange + +- Exchange + +- HashAggregate +- HashAggregate - +- HashAggregate - +- CometNativeColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Union - :- 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 - : :- 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 + +- Exchange + +- HashAggregate + +- Union + :- Project + : +- BroadcastHashJoin + : :- HashAggregate + : : +- Exchange + : : +- 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 + : +- Exchange + : +- 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 + : :- BroadcastExchange + : : +- HashAggregate + : : +- Exchange + : : +- 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 + : +- Exchange + : +- 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 + : +- Exchange : +- 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 + : +- 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 + +- Exchange +- 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 + +- 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 -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 90 out of 332 eligible operators (27%). Final plan contains 52 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_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q77a.native_iceberg_compat/extended.txt index 3578f807aa..82e3d521e5 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q77a.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q77a.native_iceberg_compat/extended.txt @@ -5,351 +5,346 @@ CometNativeColumnarToRow +- HashAggregate +- Union :- HashAggregate - : +- CometNativeColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Union - : :- CometNativeColumnarToRow - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometHashAggregate - : : : +- CometExchange - : : : +- CometHashAggregate - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - : : +- CometBroadcastExchange - : : +- CometHashAggregate - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns - : : : : +- ReusedSubquery - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - : :- Project - : : +- BroadcastNestedLoopJoin [COMET: BroadcastNestedLoopJoin is not supported] - : : :- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometHashAggregate - : : : +- CometExchange - : : : +- CometHashAggregate - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : +- ReusedSubquery - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometNativeColumnarToRow - : : +- CometHashAggregate - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns - : : : +- ReusedSubquery - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometNativeColumnarToRow - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometHashAggregate - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : +- ReusedSubquery - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_page - : +- CometBroadcastExchange - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns - : : : +- ReusedSubquery - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_page + : +- Exchange + : +- HashAggregate + : +- Union + : :- CometNativeColumnarToRow + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometHashAggregate + : : : +- CometExchange + : : : +- CometHashAggregate + : : : +- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometNativeColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store + : : +- CometBroadcastExchange + : : +- CometHashAggregate + : : +- CometExchange + : : +- CometHashAggregate + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns + : : : : +- ReusedSubquery + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store + : :- Project + : : +- BroadcastNestedLoopJoin [COMET: BroadcastNestedLoopJoin is not supported] + : : :- BroadcastExchange + : : : +- CometNativeColumnarToRow + : : : +- CometHashAggregate + : : : +- CometExchange + : : : +- CometHashAggregate + : : : +- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales + : : : : +- ReusedSubquery + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- CometNativeColumnarToRow + : : +- CometHashAggregate + : : +- CometExchange + : : +- CometHashAggregate + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns + : : : +- ReusedSubquery + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- CometNativeColumnarToRow + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometHashAggregate + : : +- CometExchange + : : +- CometHashAggregate + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales + : : : : +- ReusedSubquery + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_page + : +- CometBroadcastExchange + : +- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns + : : : +- ReusedSubquery + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_page :- HashAggregate - : +- CometNativeColumnarToRow - : +- CometColumnarExchange + : +- Exchange + : +- HashAggregate : +- HashAggregate - : +- HashAggregate - : +- CometNativeColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Union - : :- CometNativeColumnarToRow - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometHashAggregate - : : : +- CometExchange - : : : +- CometHashAggregate - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - : : +- CometBroadcastExchange - : : +- CometHashAggregate - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns - : : : : +- ReusedSubquery - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - : :- Project - : : +- BroadcastNestedLoopJoin [COMET: BroadcastNestedLoopJoin is not supported] - : : :- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometHashAggregate - : : : +- CometExchange - : : : +- CometHashAggregate - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : +- ReusedSubquery - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometNativeColumnarToRow - : : +- CometHashAggregate - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns - : : : +- ReusedSubquery - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometNativeColumnarToRow - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometHashAggregate - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : +- ReusedSubquery - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_page - : +- CometBroadcastExchange + : +- Exchange + : +- HashAggregate + : +- Union + : :- CometNativeColumnarToRow + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometHashAggregate + : : : +- CometExchange + : : : +- CometHashAggregate + : : : +- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometNativeColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store + : : +- CometBroadcastExchange + : : +- CometHashAggregate + : : +- CometExchange + : : +- CometHashAggregate + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns + : : : : +- ReusedSubquery + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store + : :- Project + : : +- BroadcastNestedLoopJoin [COMET: BroadcastNestedLoopJoin is not supported] + : : :- BroadcastExchange + : : : +- CometNativeColumnarToRow + : : : +- CometHashAggregate + : : : +- CometExchange + : : : +- CometHashAggregate + : : : +- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales + : : : : +- ReusedSubquery + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- CometNativeColumnarToRow + : : +- CometHashAggregate + : : +- CometExchange + : : +- CometHashAggregate + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns + : : : +- ReusedSubquery + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- CometNativeColumnarToRow + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometHashAggregate + : : +- CometExchange + : : +- CometHashAggregate + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales + : : : : +- ReusedSubquery + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_page + : +- CometBroadcastExchange + : +- CometHashAggregate + : +- CometExchange : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns - : : : +- ReusedSubquery - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_page + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns + : : : +- ReusedSubquery + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_page +- HashAggregate - +- CometNativeColumnarToRow - +- CometColumnarExchange + +- Exchange + +- HashAggregate +- HashAggregate - +- HashAggregate - +- CometNativeColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Union - :- CometNativeColumnarToRow - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometHashAggregate - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - : +- CometBroadcastExchange - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns - : : : +- ReusedSubquery - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - :- Project - : +- BroadcastNestedLoopJoin [COMET: BroadcastNestedLoopJoin is not supported] - : :- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometHashAggregate - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : +- ReusedSubquery - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometNativeColumnarToRow - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns - : : +- ReusedSubquery - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - +- CometNativeColumnarToRow - +- CometProject - +- CometBroadcastHashJoin - :- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : +- ReusedSubquery - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_page - +- CometBroadcastExchange + +- Exchange + +- HashAggregate + +- Union + :- CometNativeColumnarToRow + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometHashAggregate + : : +- CometExchange + : : +- CometHashAggregate + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometNativeColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store + : +- CometBroadcastExchange + : +- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns + : : : +- ReusedSubquery + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store + :- Project + : +- BroadcastNestedLoopJoin [COMET: BroadcastNestedLoopJoin is not supported] + : :- BroadcastExchange + : : +- CometNativeColumnarToRow + : : +- CometHashAggregate + : : +- CometExchange + : : +- CometHashAggregate + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales + : : : +- ReusedSubquery + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- CometNativeColumnarToRow + : +- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns + : : +- ReusedSubquery + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + +- CometNativeColumnarToRow + +- CometProject + +- CometBroadcastHashJoin + :- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales + : : : +- ReusedSubquery + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_page + +- CometBroadcastExchange + +- CometHashAggregate + +- CometExchange +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns - : : +- ReusedSubquery - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - +- CometBroadcastExchange - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_page + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns + : : +- ReusedSubquery + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + +- CometBroadcastExchange + +- CometFilter + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_page -Comet accelerated 287 out of 332 eligible operators (86%). Final plan contains 21 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 282 out of 332 eligible operators (84%). Final plan contains 16 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..bdd904b098 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 @@ -9,93 +9,88 @@ TakeOrderedAndProject +- 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 + : +- Exchange + : +- 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 + : +- Exchange + : +- HashAggregate : +- 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 + : +- Exchange + : +- 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 + +- Exchange + +- HashAggregate +- 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 + +- Exchange + +- 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 -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 31 out of 81 eligible operators (38%). 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-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..4d3c625e71 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 @@ -7,31 +7,30 @@ 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 + +- Exchange + +- 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 -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 13 out of 28 eligible operators (46%). 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/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..07ef74ce3d 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,64 @@ 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 + +- Exchange + +- 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 -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 20 out of 52 eligible operators (38%). 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/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..00d2b3b0d7 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 @@ -6,100 +6,96 @@ TakeOrderedAndProject : :- 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 + : : : +- Exchange + : : : +- 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 + : : +- Exchange + : : +- 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 + : +- Exchange + : +- 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 + +- Exchange + +- 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 -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 24 out of 85 eligible operators (28%). 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/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..79e4955126 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 @@ -5,31 +5,30 @@ TakeOrderedAndProject +- 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 + +- Exchange + +- 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 -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 11 out of 27 eligible operators (40%). 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/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..30eef295ec 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 @@ -3,406 +3,403 @@ TakeOrderedAndProject :- 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 + : : +- Exchange + : : +- 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 + : +- Exchange + : +- 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 + +- Exchange + +- 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 -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 125 out of 337 eligible operators (37%). Final plan contains 66 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..0245ef1d1c 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 @@ -5,2775 +5,2746 @@ CometNativeColumnarToRow +- 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 - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- Exchange + : +- HashAggregate + : +- Union + : :- Filter + : : : +- Subquery + : : : +- HashAggregate + : : : +- Exchange + : : : +- 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 + : : +- Exchange + : : +- 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 + : : +- Exchange + : : +- 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 + : +- Exchange + : +- 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 + : +- Exchange + : +- HashAggregate : +- 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 + : +- Exchange + : +- HashAggregate + : +- Union + : :- Filter + : : : +- Subquery + : : : +- HashAggregate + : : : +- Exchange + : : : +- 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 + : : +- Exchange + : : +- 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 + : : +- Exchange + : : +- 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 + : +- Exchange : +- 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 + : +- 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 + : +- Exchange + : +- HashAggregate : +- 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 + : +- Exchange + : +- HashAggregate + : +- Union + : :- Filter + : : : +- Subquery + : : : +- HashAggregate + : : : +- Exchange + : : : +- 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 + : : +- Exchange + : : +- 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 + : : +- Exchange + : : +- 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 + : +- Exchange : +- 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 + : +- 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 + : +- Exchange + : +- HashAggregate : +- 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 + : +- Exchange + : +- HashAggregate + : +- Union + : :- Filter + : : : +- Subquery + : : : +- HashAggregate + : : : +- Exchange + : : : +- 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 + : : +- Exchange + : : +- 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 + : : +- Exchange + : : +- 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 + : +- Exchange : +- 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 + : +- 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 + +- Exchange + +- HashAggregate +- 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 + +- Exchange + +- HashAggregate + +- Union + :- Filter + : : +- Subquery + : : +- HashAggregate + : : +- Exchange + : : +- 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 + : +- Exchange + : +- 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 + : +- Exchange + : +- 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 + +- Exchange +- 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 + +- 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 -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 813 out of 2302 eligible operators (35%). Final plan contains 446 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..1a1a476899 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,248 @@ 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 + : +- Exchange + : +- 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 + : +- Exchange + : +- 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 + : +- Exchange + : +- 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 + : +- Exchange + : +- 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 + +- Exchange + +- 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 -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 87 out of 210 eligible operators (41%). 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/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..74b85d1536 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 @@ -5,31 +5,30 @@ TakeOrderedAndProject +- 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 + +- Exchange + +- 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 -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 11 out of 27 eligible operators (40%). 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/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..bf53cec292 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 @@ -1,36 +1,35 @@ TakeOrderedAndProject +- HashAggregate - +- CometNativeColumnarToRow - +- CometColumnarExchange - +- 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 - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.item - +- BroadcastExchange - +- CometNativeColumnarToRow - +- CometNativeScan parquet spark_catalog.default.warehouse + +- Exchange + +- 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 + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.item + +- BroadcastExchange + +- 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 10 out of 28 eligible operators (35%). 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/q22.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q22.native_iceberg_compat/extended.txt index f642ebc712..136315f3db 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q22.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q22.native_iceberg_compat/extended.txt @@ -1,34 +1,33 @@ TakeOrderedAndProject +- HashAggregate - +- CometNativeColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Expand - +- Project - +- BroadcastNestedLoopJoin [COMET: BroadcastNestedLoopJoin is not supported] - :- CometNativeColumnarToRow - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.inventory - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - +- BroadcastExchange - +- CometNativeColumnarToRow - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.warehouse + +- Exchange + +- HashAggregate + +- Expand + +- Project + +- BroadcastNestedLoopJoin [COMET: BroadcastNestedLoopJoin is not supported] + :- CometNativeColumnarToRow + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.inventory + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometNativeColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + +- BroadcastExchange + +- CometNativeColumnarToRow + +- CometScan [native_iceberg_compat] 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 18 out of 28 eligible operators (64%). 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 eaa5438d91..508e7619dc 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 @@ -3,8 +3,42 @@ TakeOrderedAndProject :- HashAggregate : +- HashAggregate : +- HashAggregate - : +- CometNativeColumnarToRow - : +- CometColumnarExchange + : +- Exchange + : +- 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 + : +- Exchange + : +- HashAggregate + : +- HashAggregate + : +- Exchange : +- HashAggregate : +- Project : +- BroadcastHashJoin @@ -36,152 +70,109 @@ TakeOrderedAndProject : +- 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 + : +- Exchange + : +- 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 + : +- Exchange + : +- 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 + : +- Exchange + : +- 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 + : +- Exchange + : +- 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 + +- Exchange + +- 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 + +- Exchange + +- 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 -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 55 out of 151 eligible operators (36%). 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/q24.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q24.native_datafusion/extended.txt index 4c22965f0f..8e67cf3e18 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q24.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q24.native_datafusion/extended.txt @@ -4,96 +4,92 @@ CometNativeColumnarToRow +- Filter : +- Subquery : +- HashAggregate - : +- CometNativeColumnarToRow - : +- CometColumnarExchange + : +- Exchange + : +- HashAggregate : +- HashAggregate - : +- HashAggregate - : +- CometNativeColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin [COMET: Comet is not compatible with Spark for case conversion in locale-specific cases. Set spark.comet.caseConversion.enabled=true to enable it anyway.] - : :- CometNativeColumnarToRow - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometSortMergeJoin - : : : : : :- CometSort - : : : : : : +- CometExchange - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : : : +- CometSort - : : : : : +- CometExchange - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.store_returns - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.store - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.item - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.customer - : +- BroadcastExchange - : +- CometNativeColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.customer_address + : +- Exchange + : +- HashAggregate + : +- Project + : +- BroadcastHashJoin [COMET: Comet is not compatible with Spark for case conversion in locale-specific cases. Set spark.comet.caseConversion.enabled=true to enable it anyway.] + : :- CometNativeColumnarToRow + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometProject + : : : : : +- CometSortMergeJoin + : : : : : :- CometSort + : : : : : : +- CometExchange + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : : +- CometSort + : : : : : +- CometExchange + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.store_returns + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.store + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.item + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.customer + : +- BroadcastExchange + : +- CometNativeColumnarToRow + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.customer_address +- HashAggregate - +- CometNativeColumnarToRow - +- CometColumnarExchange + +- Exchange + +- HashAggregate +- HashAggregate - +- HashAggregate - +- CometNativeColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Project - +- BroadcastHashJoin [COMET: Comet is not compatible with Spark for case conversion in locale-specific cases. Set spark.comet.caseConversion.enabled=true to enable it anyway.] - :- CometNativeColumnarToRow - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometSortMergeJoin - : : : : :- CometSort - : : : : : +- CometExchange - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : : +- CometSort - : : : : +- CometExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.store_returns - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.store - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.item - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.customer - +- BroadcastExchange - +- CometNativeColumnarToRow - +- CometProject - +- CometFilter - +- CometNativeScan parquet spark_catalog.default.customer_address + +- Exchange + +- HashAggregate + +- Project + +- BroadcastHashJoin [COMET: Comet is not compatible with Spark for case conversion in locale-specific cases. Set spark.comet.caseConversion.enabled=true to enable it anyway.] + :- CometNativeColumnarToRow + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometSortMergeJoin + : : : : :- CometSort + : : : : : +- CometExchange + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : +- CometSort + : : : : +- CometExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.store_returns + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.store + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.item + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.customer + +- BroadcastExchange + +- CometNativeColumnarToRow + +- CometProject + +- CometFilter + +- CometNativeScan parquet spark_catalog.default.customer_address -Comet accelerated 72 out of 88 eligible operators (81%). Final plan contains 9 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 68 out of 88 eligible operators (77%). 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/q24.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q24.native_iceberg_compat/extended.txt index 5d8bc6a064..4488b9b0ff 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q24.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q24.native_iceberg_compat/extended.txt @@ -4,96 +4,92 @@ CometNativeColumnarToRow +- Filter : +- Subquery : +- HashAggregate - : +- CometNativeColumnarToRow - : +- CometColumnarExchange + : +- Exchange + : +- HashAggregate : +- HashAggregate - : +- HashAggregate - : +- CometNativeColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin [COMET: Comet is not compatible with Spark for case conversion in locale-specific cases. Set spark.comet.caseConversion.enabled=true to enable it anyway.] - : :- CometNativeColumnarToRow - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometSortMergeJoin - : : : : : :- CometSort - : : : : : : +- CometExchange - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : +- CometSort - : : : : : +- CometExchange - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer - : +- BroadcastExchange - : +- CometNativeColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address + : +- Exchange + : +- HashAggregate + : +- Project + : +- BroadcastHashJoin [COMET: Comet is not compatible with Spark for case conversion in locale-specific cases. Set spark.comet.caseConversion.enabled=true to enable it anyway.] + : :- CometNativeColumnarToRow + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometProject + : : : : : +- CometSortMergeJoin + : : : : : :- CometSort + : : : : : : +- CometExchange + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales + : : : : : +- CometSort + : : : : : +- CometExchange + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer + : +- BroadcastExchange + : +- CometNativeColumnarToRow + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address +- HashAggregate - +- CometNativeColumnarToRow - +- CometColumnarExchange + +- Exchange + +- HashAggregate +- HashAggregate - +- HashAggregate - +- CometNativeColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Project - +- BroadcastHashJoin [COMET: Comet is not compatible with Spark for case conversion in locale-specific cases. Set spark.comet.caseConversion.enabled=true to enable it anyway.] - :- CometNativeColumnarToRow - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometSortMergeJoin - : : : : :- CometSort - : : : : : +- CometExchange - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : +- CometSort - : : : : +- CometExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer - +- BroadcastExchange - +- CometNativeColumnarToRow - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address + +- Exchange + +- HashAggregate + +- Project + +- BroadcastHashJoin [COMET: Comet is not compatible with Spark for case conversion in locale-specific cases. Set spark.comet.caseConversion.enabled=true to enable it anyway.] + :- CometNativeColumnarToRow + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometSortMergeJoin + : : : : :- CometSort + : : : : : +- CometExchange + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales + : : : : +- CometSort + : : : : +- CometExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer + +- BroadcastExchange + +- CometNativeColumnarToRow + +- CometProject + +- CometFilter + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address -Comet accelerated 72 out of 88 eligible operators (81%). Final plan contains 9 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 68 out of 88 eligible operators (77%). 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/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..793de3abde 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,113 @@ 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 + : +- Exchange + : +- 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 + : +- Exchange + : +- 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 + +- Exchange + +- 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 -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 38 out of 95 eligible operators (40%). Final plan contains 16 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..575c853e76 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 @@ -5,43 +5,42 @@ CometNativeColumnarToRow +- 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 + : +- Exchange + : +- 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 -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 17 out of 37 eligible operators (45%). 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/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..c8a9740a02 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 @@ -1,67 +1,66 @@ TakeOrderedAndProject +- HashAggregate - +- CometNativeColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- 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 - : : : : +- 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 - : : : +- 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 - : : +- 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 + +- Exchange + +- HashAggregate + +- Project + +- BroadcastHashJoin + :- Project + : +- BroadcastHashJoin + : :- 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 + : : : : +- 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 + : : : +- 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 + : : +- 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 -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 20 out of 54 eligible operators (37%). 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/q35.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q35.native_iceberg_compat/extended.txt index c5125b1861..72960fc267 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q35.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q35.native_iceberg_compat/extended.txt @@ -1,63 +1,62 @@ TakeOrderedAndProject +- HashAggregate - +- CometNativeColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- Project - : : +- Filter - : : +- BroadcastHashJoin - : : :- BroadcastHashJoin [COMET: Unsupported join type ExistenceJoin(exists#1)] - : : : :- CometNativeColumnarToRow - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : +- ReusedSubquery - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : +- ReusedSubquery - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- CometNativeColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address - +- BroadcastExchange - +- CometNativeColumnarToRow - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics + +- Exchange + +- HashAggregate + +- Project + +- BroadcastHashJoin + :- Project + : +- BroadcastHashJoin + : :- Project + : : +- Filter + : : +- BroadcastHashJoin + : : :- BroadcastHashJoin [COMET: Unsupported join type ExistenceJoin(exists#1)] + : : : :- CometNativeColumnarToRow + : : : : +- CometBroadcastHashJoin + : : : : :- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometNativeColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : +- BroadcastExchange + : : : +- CometNativeColumnarToRow + : : : +- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales + : : : : +- ReusedSubquery + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- BroadcastExchange + : : +- CometNativeColumnarToRow + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales + : : : +- ReusedSubquery + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- BroadcastExchange + : +- CometNativeColumnarToRow + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address + +- BroadcastExchange + +- CometNativeColumnarToRow + +- CometProject + +- CometFilter + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics -Comet accelerated 35 out of 54 eligible operators (64%). Final plan contains 7 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 34 out of 54 eligible operators (62%). 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 9091536e96..07ef74ce3d 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,64 @@ 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 + +- Exchange + +- 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 -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 20 out of 52 eligible operators (38%). 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/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..380076587c 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 @@ -9,114 +9,109 @@ TakeOrderedAndProject +- 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 + : +- Exchange + : +- 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 + : +- Exchange + : +- HashAggregate : +- 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 + : +- Exchange + : +- 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 + +- Exchange + +- HashAggregate +- 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 + +- Exchange + +- 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 -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 40 out of 99 eligible operators (40%). Final plan contains 16 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..e347d0f433 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 @@ -12,36 +12,35 @@ TakeOrderedAndProject : : +- 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 + : : +- Exchange + : : +- 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 : +- 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).] @@ -49,36 +48,35 @@ TakeOrderedAndProject : +- 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 + : +- Exchange + : +- 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 +- 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).] @@ -86,35 +84,34 @@ TakeOrderedAndProject +- 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 + +- Exchange + +- 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 -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 33 out of 97 eligible operators (34%). 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/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..d19eb04caf 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 @@ -13,33 +13,32 @@ 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 + : +- Exchange + : +- 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 :- Project : +- Filter : +- Window @@ -49,28 +48,27 @@ 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 + : +- Exchange + : +- 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 +- Project +- Filter +- Window @@ -80,27 +78,26 @@ 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 + +- Exchange + +- 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 -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 30 out of 87 eligible operators (34%). 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/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..7bccd55531 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 @@ -14,111 +14,105 @@ TakeOrderedAndProject : :- CometSort : : +- CometColumnarExchange : : +- HashAggregate - : : +- CometNativeColumnarToRow - : : +- CometColumnarExchange - : : +- HashAggregate - : : +- Project - : : +- BroadcastHashJoin - : : :- 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 - : : +- 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 + : : +- Exchange + : : +- HashAggregate + : : +- Project + : : +- BroadcastHashJoin + : : :- 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 + : : : +- Exchange + : : : +- 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 + : : +- 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 + : : +- Exchange + : : +- 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 : +- CometSort : +- CometColumnarExchange : +- HashAggregate - : +- CometNativeColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- 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 - : +- 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 + : +- Exchange + : +- HashAggregate + : +- Project + : +- BroadcastHashJoin + : :- 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 + : : +- Exchange + : : +- 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 + : +- 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 + : +- Exchange + : +- 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 +- 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).] @@ -131,110 +125,104 @@ TakeOrderedAndProject :- CometSort : +- CometColumnarExchange : +- HashAggregate - : +- CometNativeColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- 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 - : +- 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 + : +- Exchange + : +- HashAggregate + : +- Project + : +- BroadcastHashJoin + : :- 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 + : : +- Exchange + : : +- 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 + : +- 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 + : +- Exchange + : +- 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 +- CometSort +- CometColumnarExchange +- HashAggregate - +- CometNativeColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Project - +- BroadcastHashJoin - :- 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 - +- 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 + +- Exchange + +- HashAggregate + +- Project + +- BroadcastHashJoin + :- 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 + : +- Exchange + : +- 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 + +- 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 + +- Exchange + +- 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 -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 70 out of 196 eligible operators (35%). Final plan contains 30 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_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q51a.native_iceberg_compat/extended.txt index 6560795de1..190830204f 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q51a.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q51a.native_iceberg_compat/extended.txt @@ -14,99 +14,97 @@ TakeOrderedAndProject : :- CometSort : : +- CometColumnarExchange : : +- HashAggregate - : : +- CometNativeColumnarToRow - : : +- CometColumnarExchange - : : +- HashAggregate - : : +- Project - : : +- BroadcastHashJoin - : : :- 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 - : : : +- CometExchange - : : : +- CometHashAggregate - : : : +- CometExchange - : : : +- CometHashAggregate - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] 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 - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- Exchange + : : +- HashAggregate + : : +- Project + : : +- BroadcastHashJoin + : : :- 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 + : : : +- CometExchange + : : : +- CometHashAggregate + : : : +- CometExchange + : : : +- CometHashAggregate + : : : +- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometNativeColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] 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 + : : +- CometExchange + : : +- CometHashAggregate + : : +- CometExchange + : : +- CometHashAggregate + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometNativeColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim : +- CometSort : +- CometColumnarExchange : +- HashAggregate - : +- CometNativeColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- 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 - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : +- ReusedSubquery - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] 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 - : +- CometExchange - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : +- ReusedSubquery - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- Exchange + : +- HashAggregate + : +- Project + : +- BroadcastHashJoin + : :- 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 + : : +- CometExchange + : : +- CometHashAggregate + : : +- CometExchange + : : +- CometHashAggregate + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales + : : : +- ReusedSubquery + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] 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 + : +- CometExchange + : +- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales + : : +- ReusedSubquery + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] 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).] @@ -119,98 +117,96 @@ TakeOrderedAndProject :- CometSort : +- CometColumnarExchange : +- HashAggregate - : +- CometNativeColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- 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 - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] 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 - : +- CometExchange - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- Exchange + : +- HashAggregate + : +- Project + : +- BroadcastHashJoin + : :- 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 + : : +- CometExchange + : : +- CometHashAggregate + : : +- CometExchange + : : +- CometHashAggregate + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometNativeColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] 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 + : +- CometExchange + : +- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales + : : +- SubqueryBroadcast + : : +- BroadcastExchange + : : +- CometNativeColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim +- CometSort +- CometColumnarExchange +- HashAggregate - +- CometNativeColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Project - +- BroadcastHashJoin - :- 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 - : +- CometExchange - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : +- ReusedSubquery - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] 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 - +- CometExchange - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : +- ReusedSubquery - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + +- Exchange + +- HashAggregate + +- Project + +- BroadcastHashJoin + :- 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 + : +- CometExchange + : +- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales + : : +- ReusedSubquery + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] 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 + +- CometExchange + +- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometProject + +- CometBroadcastHashJoin + :- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales + : +- ReusedSubquery + +- CometBroadcastExchange + +- CometProject + +- CometFilter + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Comet accelerated 138 out of 196 eligible operators (70%). Final plan contains 18 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 134 out of 196 eligible operators (68%). 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 66c510aaa0..50dc55f674 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 @@ -12,36 +12,35 @@ TakeOrderedAndProject : : +- 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 + : : +- Exchange + : : +- 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 : +- 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).] @@ -49,36 +48,35 @@ TakeOrderedAndProject : +- 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 + : +- Exchange + : +- 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 +- 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).] @@ -86,35 +84,34 @@ TakeOrderedAndProject +- 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 + +- Exchange + +- 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 -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 33 out of 97 eligible operators (34%). 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/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..858a24872a 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 @@ -5,318 +5,304 @@ CometNativeColumnarToRow +- 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 - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.web_site + : +- Exchange + : +- HashAggregate + : +- Union + : :- HashAggregate + : : +- Exchange + : : +- 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 + : : +- Exchange + : : +- 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 + : +- Exchange + : +- 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 + : +- Exchange + : +- HashAggregate : +- HashAggregate - : +- HashAggregate - : +- CometNativeColumnarToRow - : +- CometColumnarExchange + : +- Exchange + : +- HashAggregate + : +- Union + : :- HashAggregate + : : +- Exchange + : : +- 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 + : : +- Exchange + : : +- 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 - : +- 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 + : +- Exchange : +- 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 + : +- 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 + +- Exchange + +- HashAggregate +- HashAggregate - +- HashAggregate - +- CometNativeColumnarToRow - +- CometColumnarExchange + +- Exchange + +- HashAggregate + +- Union + :- HashAggregate + : +- Exchange + : +- 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 + : +- Exchange + : +- 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 - +- 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 + +- Exchange +- 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 + +- 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 -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 75 out of 263 eligible operators (28%). Final plan contains 43 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..bdf9399f8b 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,69 @@ 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 + +- Exchange + +- 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 -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 38 out of 60 eligible operators (63%). 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/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..245dc7a4d5 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 @@ -9,342 +9,325 @@ TakeOrderedAndProject +- 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 + : +- Exchange + : +- 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 + : +- Exchange + : +- HashAggregate : +- 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 + : +- Exchange + : +- 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 + : +- Exchange + : +- HashAggregate : +- 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 + : +- Exchange + : +- 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 + : +- Exchange + : +- HashAggregate : +- 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 + : +- Exchange + : +- 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 + : +- Exchange + : +- HashAggregate : +- 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 + : +- Exchange + : +- 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 + : +- Exchange + : +- HashAggregate : +- 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 + : +- Exchange + : +- 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 + : +- Exchange + : +- HashAggregate : +- 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 + : +- Exchange + : +- 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 + : +- Exchange + : +- HashAggregate : +- 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 + : +- Exchange + : +- 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 + +- Exchange + +- HashAggregate +- 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 + +- Exchange + +- 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 -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 110 out of 285 eligible operators (38%). Final plan contains 46 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..70155e92ca 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 @@ -9,180 +9,172 @@ TakeOrderedAndProject +- HashAggregate +- Union :- 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 - : +- Project - : +- BroadcastHashJoin - : :- CometNativeColumnarToRow - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.store - : +- BroadcastExchange - : +- 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 + : +- Exchange + : +- 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 + : +- Project + : +- BroadcastHashJoin + : :- CometNativeColumnarToRow + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.store + : +- BroadcastExchange + : +- Project + : +- Filter + : +- Window + : +- WindowGroupLimit + : +- Sort + : +- HashAggregate + : +- Exchange + : +- 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 :- HashAggregate - : +- CometNativeColumnarToRow - : +- CometColumnarExchange + : +- Exchange + : +- HashAggregate : +- 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 - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- Project - : +- BroadcastHashJoin - : :- CometNativeColumnarToRow - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.store - : +- BroadcastExchange - : +- Project - : +- Filter - : +- Window - : +- WindowGroupLimit - : +- Sort + : +- Exchange + : +- 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 + : +- Project + : +- BroadcastHashJoin + : :- CometNativeColumnarToRow + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.store + : +- BroadcastExchange + : +- Project + : +- Filter + : +- Window + : +- WindowGroupLimit + : +- Sort + : +- HashAggregate + : +- Exchange : +- 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 + : +- 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 +- HashAggregate - +- CometNativeColumnarToRow - +- CometColumnarExchange + +- Exchange + +- HashAggregate +- 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 - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.date_dim - +- BroadcastExchange - +- Project - +- BroadcastHashJoin - :- CometNativeColumnarToRow - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.store - +- BroadcastExchange - +- Project - +- Filter - +- Window - +- WindowGroupLimit - +- Sort + +- Exchange + +- 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 + +- Project + +- BroadcastHashJoin + :- CometNativeColumnarToRow + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.store + +- BroadcastExchange + +- Project + +- Filter + +- Window + +- WindowGroupLimit + +- Sort + +- HashAggregate + +- Exchange +- 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 + +- 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 -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 46 out of 156 eligible operators (29%). 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/q70a.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q70a.native_iceberg_compat/extended.txt index b21df154e6..ff6d42786b 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q70a.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q70a.native_iceberg_compat/extended.txt @@ -9,168 +9,163 @@ TakeOrderedAndProject +- HashAggregate +- Union :- HashAggregate - : +- CometNativeColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- CometNativeColumnarToRow - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- Project - : +- BroadcastHashJoin - : :- CometNativeColumnarToRow - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - : +- BroadcastExchange - : +- Project - : +- Filter - : +- Window - : +- WindowGroupLimit [COMET: WindowGroupLimit is not supported] - : +- CometNativeColumnarToRow - : +- CometSort - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : +- ReusedSubquery - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- Exchange + : +- HashAggregate + : +- Project + : +- BroadcastHashJoin + : :- CometNativeColumnarToRow + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometNativeColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- BroadcastExchange + : +- Project + : +- BroadcastHashJoin + : :- CometNativeColumnarToRow + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store + : +- BroadcastExchange + : +- Project + : +- Filter + : +- Window + : +- WindowGroupLimit [COMET: WindowGroupLimit is not supported] + : +- CometNativeColumnarToRow + : +- CometSort + : +- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales + : : : +- ReusedSubquery + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim :- HashAggregate - : +- CometNativeColumnarToRow - : +- CometColumnarExchange + : +- Exchange + : +- HashAggregate : +- HashAggregate - : +- HashAggregate - : +- CometNativeColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- CometNativeColumnarToRow - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- Project - : +- BroadcastHashJoin - : :- CometNativeColumnarToRow - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - : +- BroadcastExchange - : +- Project - : +- Filter - : +- Window - : +- WindowGroupLimit [COMET: WindowGroupLimit is not supported] - : +- CometNativeColumnarToRow - : +- CometSort + : +- Exchange + : +- HashAggregate + : +- Project + : +- BroadcastHashJoin + : :- CometNativeColumnarToRow + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometNativeColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- BroadcastExchange + : +- Project + : +- BroadcastHashJoin + : :- CometNativeColumnarToRow + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store + : +- BroadcastExchange + : +- Project + : +- Filter + : +- Window + : +- WindowGroupLimit [COMET: WindowGroupLimit is not supported] + : +- CometNativeColumnarToRow + : +- CometSort + : +- CometHashAggregate + : +- CometExchange : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : +- ReusedSubquery - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales + : : : +- ReusedSubquery + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim +- HashAggregate - +- CometNativeColumnarToRow - +- CometColumnarExchange + +- Exchange + +- HashAggregate +- HashAggregate - +- HashAggregate - +- CometNativeColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Project - +- BroadcastHashJoin - :- CometNativeColumnarToRow - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - +- BroadcastExchange - +- Project - +- BroadcastHashJoin - :- CometNativeColumnarToRow - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - +- BroadcastExchange - +- Project - +- Filter - +- Window - +- WindowGroupLimit [COMET: WindowGroupLimit is not supported] - +- CometNativeColumnarToRow - +- CometSort + +- Exchange + +- HashAggregate + +- Project + +- BroadcastHashJoin + :- CometNativeColumnarToRow + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales + : : +- SubqueryBroadcast + : : +- BroadcastExchange + : : +- CometNativeColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + +- BroadcastExchange + +- Project + +- BroadcastHashJoin + :- CometNativeColumnarToRow + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store + +- BroadcastExchange + +- Project + +- Filter + +- Window + +- WindowGroupLimit [COMET: WindowGroupLimit is not supported] + +- CometNativeColumnarToRow + +- CometSort + +- CometHashAggregate + +- CometExchange +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : +- ReusedSubquery - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales + : : +- ReusedSubquery + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store + +- CometBroadcastExchange + +- CometProject + +- CometFilter + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Comet accelerated 102 out of 156 eligible operators (65%). Final plan contains 18 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 97 out of 156 eligible operators (62%). 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-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..00d2b3b0d7 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 @@ -6,100 +6,96 @@ TakeOrderedAndProject : :- 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 + : : : +- Exchange + : : : +- 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 + : : +- Exchange + : : +- 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 + : +- Exchange + : +- 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 + +- Exchange + +- 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 -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 24 out of 85 eligible operators (28%). 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/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..d11e87a30b 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 @@ -5,405 +5,382 @@ CometNativeColumnarToRow +- HashAggregate +- Union :- HashAggregate - : +- CometNativeColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Union - : :- 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 - : : :- 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 + : +- Exchange + : +- HashAggregate + : +- Union + : :- Project + : : +- BroadcastHashJoin + : : :- HashAggregate + : : : +- Exchange + : : : +- 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 + : : +- Exchange + : : +- 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 + : : :- BroadcastExchange + : : : +- HashAggregate + : : : +- Exchange + : : : +- 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 + : : +- Exchange + : : +- 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 + : : +- Exchange + : : +- 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 + : +- Exchange + : +- 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 :- HashAggregate - : +- CometNativeColumnarToRow - : +- CometColumnarExchange + : +- Exchange + : +- HashAggregate : +- HashAggregate - : +- HashAggregate - : +- CometNativeColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Union - : :- 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 - : : :- 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 + : +- Exchange + : +- HashAggregate + : +- Union + : :- Project + : : +- BroadcastHashJoin + : : :- HashAggregate + : : : +- Exchange + : : : +- 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 + : : +- Exchange + : : +- 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 + : : :- BroadcastExchange + : : : +- HashAggregate + : : : +- Exchange + : : : +- 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 + : : +- Exchange + : : +- 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 + : : +- Exchange : : +- 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 + : : +- 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 + : +- Exchange : +- 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 + : +- 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 +- HashAggregate - +- CometNativeColumnarToRow - +- CometColumnarExchange + +- Exchange + +- HashAggregate +- HashAggregate - +- HashAggregate - +- CometNativeColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Union - :- 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 - : :- 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 + +- Exchange + +- HashAggregate + +- Union + :- Project + : +- BroadcastHashJoin + : :- HashAggregate + : : +- Exchange + : : +- 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 + : +- Exchange + : +- 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 + : :- BroadcastExchange + : : +- HashAggregate + : : +- Exchange + : : +- 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 + : +- Exchange + : +- 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 + : +- Exchange : +- 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 + : +- 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 + +- Exchange +- 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 + +- 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 -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 90 out of 332 eligible operators (27%). Final plan contains 52 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_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q77a.native_iceberg_compat/extended.txt index 3578f807aa..82e3d521e5 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q77a.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q77a.native_iceberg_compat/extended.txt @@ -5,351 +5,346 @@ CometNativeColumnarToRow +- HashAggregate +- Union :- HashAggregate - : +- CometNativeColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Union - : :- CometNativeColumnarToRow - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometHashAggregate - : : : +- CometExchange - : : : +- CometHashAggregate - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - : : +- CometBroadcastExchange - : : +- CometHashAggregate - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns - : : : : +- ReusedSubquery - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - : :- Project - : : +- BroadcastNestedLoopJoin [COMET: BroadcastNestedLoopJoin is not supported] - : : :- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometHashAggregate - : : : +- CometExchange - : : : +- CometHashAggregate - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : +- ReusedSubquery - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometNativeColumnarToRow - : : +- CometHashAggregate - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns - : : : +- ReusedSubquery - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometNativeColumnarToRow - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometHashAggregate - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : +- ReusedSubquery - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_page - : +- CometBroadcastExchange - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns - : : : +- ReusedSubquery - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_page + : +- Exchange + : +- HashAggregate + : +- Union + : :- CometNativeColumnarToRow + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometHashAggregate + : : : +- CometExchange + : : : +- CometHashAggregate + : : : +- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometNativeColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store + : : +- CometBroadcastExchange + : : +- CometHashAggregate + : : +- CometExchange + : : +- CometHashAggregate + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns + : : : : +- ReusedSubquery + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store + : :- Project + : : +- BroadcastNestedLoopJoin [COMET: BroadcastNestedLoopJoin is not supported] + : : :- BroadcastExchange + : : : +- CometNativeColumnarToRow + : : : +- CometHashAggregate + : : : +- CometExchange + : : : +- CometHashAggregate + : : : +- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales + : : : : +- ReusedSubquery + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- CometNativeColumnarToRow + : : +- CometHashAggregate + : : +- CometExchange + : : +- CometHashAggregate + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns + : : : +- ReusedSubquery + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- CometNativeColumnarToRow + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometHashAggregate + : : +- CometExchange + : : +- CometHashAggregate + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales + : : : : +- ReusedSubquery + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_page + : +- CometBroadcastExchange + : +- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns + : : : +- ReusedSubquery + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_page :- HashAggregate - : +- CometNativeColumnarToRow - : +- CometColumnarExchange + : +- Exchange + : +- HashAggregate : +- HashAggregate - : +- HashAggregate - : +- CometNativeColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Union - : :- CometNativeColumnarToRow - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometHashAggregate - : : : +- CometExchange - : : : +- CometHashAggregate - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - : : +- CometBroadcastExchange - : : +- CometHashAggregate - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns - : : : : +- ReusedSubquery - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - : :- Project - : : +- BroadcastNestedLoopJoin [COMET: BroadcastNestedLoopJoin is not supported] - : : :- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometHashAggregate - : : : +- CometExchange - : : : +- CometHashAggregate - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : +- ReusedSubquery - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometNativeColumnarToRow - : : +- CometHashAggregate - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns - : : : +- ReusedSubquery - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometNativeColumnarToRow - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometHashAggregate - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : +- ReusedSubquery - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_page - : +- CometBroadcastExchange + : +- Exchange + : +- HashAggregate + : +- Union + : :- CometNativeColumnarToRow + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometHashAggregate + : : : +- CometExchange + : : : +- CometHashAggregate + : : : +- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometNativeColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store + : : +- CometBroadcastExchange + : : +- CometHashAggregate + : : +- CometExchange + : : +- CometHashAggregate + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns + : : : : +- ReusedSubquery + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store + : :- Project + : : +- BroadcastNestedLoopJoin [COMET: BroadcastNestedLoopJoin is not supported] + : : :- BroadcastExchange + : : : +- CometNativeColumnarToRow + : : : +- CometHashAggregate + : : : +- CometExchange + : : : +- CometHashAggregate + : : : +- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales + : : : : +- ReusedSubquery + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- CometNativeColumnarToRow + : : +- CometHashAggregate + : : +- CometExchange + : : +- CometHashAggregate + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns + : : : +- ReusedSubquery + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- CometNativeColumnarToRow + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometHashAggregate + : : +- CometExchange + : : +- CometHashAggregate + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales + : : : : +- ReusedSubquery + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_page + : +- CometBroadcastExchange + : +- CometHashAggregate + : +- CometExchange : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns - : : : +- ReusedSubquery - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_page + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns + : : : +- ReusedSubquery + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_page +- HashAggregate - +- CometNativeColumnarToRow - +- CometColumnarExchange + +- Exchange + +- HashAggregate +- HashAggregate - +- HashAggregate - +- CometNativeColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Union - :- CometNativeColumnarToRow - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometHashAggregate - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - : +- CometBroadcastExchange - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns - : : : +- ReusedSubquery - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - :- Project - : +- BroadcastNestedLoopJoin [COMET: BroadcastNestedLoopJoin is not supported] - : :- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometHashAggregate - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : +- ReusedSubquery - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometNativeColumnarToRow - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns - : : +- ReusedSubquery - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - +- CometNativeColumnarToRow - +- CometProject - +- CometBroadcastHashJoin - :- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : +- ReusedSubquery - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_page - +- CometBroadcastExchange + +- Exchange + +- HashAggregate + +- Union + :- CometNativeColumnarToRow + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometHashAggregate + : : +- CometExchange + : : +- CometHashAggregate + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometNativeColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store + : +- CometBroadcastExchange + : +- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns + : : : +- ReusedSubquery + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store + :- Project + : +- BroadcastNestedLoopJoin [COMET: BroadcastNestedLoopJoin is not supported] + : :- BroadcastExchange + : : +- CometNativeColumnarToRow + : : +- CometHashAggregate + : : +- CometExchange + : : +- CometHashAggregate + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales + : : : +- ReusedSubquery + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- CometNativeColumnarToRow + : +- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns + : : +- ReusedSubquery + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + +- CometNativeColumnarToRow + +- CometProject + +- CometBroadcastHashJoin + :- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales + : : : +- ReusedSubquery + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_page + +- CometBroadcastExchange + +- CometHashAggregate + +- CometExchange +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns - : : +- ReusedSubquery - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - +- CometBroadcastExchange - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_page + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns + : : +- ReusedSubquery + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + +- CometBroadcastExchange + +- CometFilter + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_page -Comet accelerated 287 out of 332 eligible operators (86%). Final plan contains 21 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 282 out of 332 eligible operators (84%). Final plan contains 16 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..bdd904b098 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 @@ -9,93 +9,88 @@ TakeOrderedAndProject +- 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 + : +- Exchange + : +- 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 + : +- Exchange + : +- HashAggregate : +- 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 + : +- Exchange + : +- 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 + +- Exchange + +- HashAggregate +- 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 + +- Exchange + +- 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 -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 31 out of 81 eligible operators (38%). 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-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..4d3c625e71 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 @@ -7,31 +7,30 @@ 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 + +- Exchange + +- 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 -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 13 out of 28 eligible operators (46%). 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/q10a.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q10a.native_datafusion/extended.txt index 9091536e96..07ef74ce3d 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,64 @@ 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 + +- Exchange + +- 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 -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 20 out of 52 eligible operators (38%). 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/q11.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q11.native_datafusion/extended.txt index 5d218998be..00d2b3b0d7 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 @@ -6,100 +6,96 @@ TakeOrderedAndProject : :- 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 + : : : +- Exchange + : : : +- 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 + : : +- Exchange + : : +- 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 + : +- Exchange + : +- 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 + +- Exchange + +- 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 -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 24 out of 85 eligible operators (28%). 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/q12.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q12.native_datafusion/extended.txt index 15648408fc..79e4955126 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 @@ -5,31 +5,30 @@ TakeOrderedAndProject +- 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 + +- Exchange + +- 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 -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 11 out of 27 eligible operators (40%). 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/q14.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14.native_datafusion/extended.txt index 7921f2fa61..8d34a30261 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 @@ -3,402 +3,399 @@ TakeOrderedAndProject :- 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 + : : +- Exchange + : : +- 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 + : +- Exchange + : +- 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 + +- Exchange + +- 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 -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 125 out of 333 eligible operators (37%). Final plan contains 66 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..0245ef1d1c 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 @@ -5,2775 +5,2746 @@ CometNativeColumnarToRow +- 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 - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- Exchange + : +- HashAggregate + : +- Union + : :- Filter + : : : +- Subquery + : : : +- HashAggregate + : : : +- Exchange + : : : +- 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 + : : +- Exchange + : : +- 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 + : : +- Exchange + : : +- 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 + : +- Exchange + : +- 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 + : +- Exchange + : +- HashAggregate : +- 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 + : +- Exchange + : +- HashAggregate + : +- Union + : :- Filter + : : : +- Subquery + : : : +- HashAggregate + : : : +- Exchange + : : : +- 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 + : : +- Exchange + : : +- 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 + : : +- Exchange + : : +- 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 + : +- Exchange : +- 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 + : +- 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 + : +- Exchange + : +- HashAggregate : +- 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 + : +- Exchange + : +- HashAggregate + : +- Union + : :- Filter + : : : +- Subquery + : : : +- HashAggregate + : : : +- Exchange + : : : +- 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 + : : +- Exchange + : : +- 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 + : : +- Exchange + : : +- 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 + : +- Exchange : +- 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 + : +- 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 + : +- Exchange + : +- HashAggregate : +- 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 + : +- Exchange + : +- HashAggregate + : +- Union + : :- Filter + : : : +- Subquery + : : : +- HashAggregate + : : : +- Exchange + : : : +- 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 + : : +- Exchange + : : +- 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 + : : +- Exchange + : : +- 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 + : +- Exchange : +- 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 + : +- 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 + +- Exchange + +- HashAggregate +- 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 + +- Exchange + +- HashAggregate + +- Union + :- Filter + : : +- Subquery + : : +- HashAggregate + : : +- Exchange + : : +- 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 + : +- Exchange + : +- 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 + : +- Exchange + : +- 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 + +- Exchange +- 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 + +- 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 -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 813 out of 2302 eligible operators (35%). Final plan contains 446 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..1a1a476899 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,248 @@ 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 + : +- Exchange + : +- 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 + : +- Exchange + : +- 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 + : +- Exchange + : +- 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 + : +- Exchange + : +- 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 + +- Exchange + +- 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 -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 87 out of 210 eligible operators (41%). 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/q20.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q20.native_datafusion/extended.txt index 6634edcbed..74b85d1536 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 @@ -5,31 +5,30 @@ TakeOrderedAndProject +- 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 + +- Exchange + +- 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 -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 11 out of 27 eligible operators (40%). 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/q22.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q22.native_datafusion/extended.txt index 108e3c13f9..bf53cec292 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 @@ -1,36 +1,35 @@ TakeOrderedAndProject +- HashAggregate - +- CometNativeColumnarToRow - +- CometColumnarExchange - +- 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 - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.item - +- BroadcastExchange - +- CometNativeColumnarToRow - +- CometNativeScan parquet spark_catalog.default.warehouse + +- Exchange + +- 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 + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.item + +- BroadcastExchange + +- 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 10 out of 28 eligible operators (35%). 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/q22.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q22.native_iceberg_compat/extended.txt index f642ebc712..136315f3db 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q22.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q22.native_iceberg_compat/extended.txt @@ -1,34 +1,33 @@ TakeOrderedAndProject +- HashAggregate - +- CometNativeColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Expand - +- Project - +- BroadcastNestedLoopJoin [COMET: BroadcastNestedLoopJoin is not supported] - :- CometNativeColumnarToRow - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.inventory - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - +- BroadcastExchange - +- CometNativeColumnarToRow - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.warehouse + +- Exchange + +- HashAggregate + +- Expand + +- Project + +- BroadcastNestedLoopJoin [COMET: BroadcastNestedLoopJoin is not supported] + :- CometNativeColumnarToRow + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.inventory + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometNativeColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + +- BroadcastExchange + +- CometNativeColumnarToRow + +- CometScan [native_iceberg_compat] 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 18 out of 28 eligible operators (64%). 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 eaa5438d91..508e7619dc 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 @@ -3,8 +3,42 @@ TakeOrderedAndProject :- HashAggregate : +- HashAggregate : +- HashAggregate - : +- CometNativeColumnarToRow - : +- CometColumnarExchange + : +- Exchange + : +- 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 + : +- Exchange + : +- HashAggregate + : +- HashAggregate + : +- Exchange : +- HashAggregate : +- Project : +- BroadcastHashJoin @@ -36,152 +70,109 @@ TakeOrderedAndProject : +- 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 + : +- Exchange + : +- 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 + : +- Exchange + : +- 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 + : +- Exchange + : +- 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 + : +- Exchange + : +- 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 + +- Exchange + +- 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 + +- Exchange + +- 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 -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 55 out of 151 eligible operators (36%). 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/q24.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q24.native_datafusion/extended.txt index 4c22965f0f..8e67cf3e18 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q24.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q24.native_datafusion/extended.txt @@ -4,96 +4,92 @@ CometNativeColumnarToRow +- Filter : +- Subquery : +- HashAggregate - : +- CometNativeColumnarToRow - : +- CometColumnarExchange + : +- Exchange + : +- HashAggregate : +- HashAggregate - : +- HashAggregate - : +- CometNativeColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin [COMET: Comet is not compatible with Spark for case conversion in locale-specific cases. Set spark.comet.caseConversion.enabled=true to enable it anyway.] - : :- CometNativeColumnarToRow - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometSortMergeJoin - : : : : : :- CometSort - : : : : : : +- CometExchange - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : : : +- CometSort - : : : : : +- CometExchange - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.store_returns - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.store - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.item - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.customer - : +- BroadcastExchange - : +- CometNativeColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.customer_address + : +- Exchange + : +- HashAggregate + : +- Project + : +- BroadcastHashJoin [COMET: Comet is not compatible with Spark for case conversion in locale-specific cases. Set spark.comet.caseConversion.enabled=true to enable it anyway.] + : :- CometNativeColumnarToRow + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometProject + : : : : : +- CometSortMergeJoin + : : : : : :- CometSort + : : : : : : +- CometExchange + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : : +- CometSort + : : : : : +- CometExchange + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.store_returns + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.store + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.item + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.customer + : +- BroadcastExchange + : +- CometNativeColumnarToRow + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.customer_address +- HashAggregate - +- CometNativeColumnarToRow - +- CometColumnarExchange + +- Exchange + +- HashAggregate +- HashAggregate - +- HashAggregate - +- CometNativeColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Project - +- BroadcastHashJoin [COMET: Comet is not compatible with Spark for case conversion in locale-specific cases. Set spark.comet.caseConversion.enabled=true to enable it anyway.] - :- CometNativeColumnarToRow - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometSortMergeJoin - : : : : :- CometSort - : : : : : +- CometExchange - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : : +- CometSort - : : : : +- CometExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.store_returns - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.store - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.item - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.customer - +- BroadcastExchange - +- CometNativeColumnarToRow - +- CometProject - +- CometFilter - +- CometNativeScan parquet spark_catalog.default.customer_address + +- Exchange + +- HashAggregate + +- Project + +- BroadcastHashJoin [COMET: Comet is not compatible with Spark for case conversion in locale-specific cases. Set spark.comet.caseConversion.enabled=true to enable it anyway.] + :- CometNativeColumnarToRow + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometSortMergeJoin + : : : : :- CometSort + : : : : : +- CometExchange + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : +- CometSort + : : : : +- CometExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.store_returns + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.store + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.item + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.customer + +- BroadcastExchange + +- CometNativeColumnarToRow + +- CometProject + +- CometFilter + +- CometNativeScan parquet spark_catalog.default.customer_address -Comet accelerated 72 out of 88 eligible operators (81%). Final plan contains 9 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 68 out of 88 eligible operators (77%). 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/q24.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q24.native_iceberg_compat/extended.txt index 5d8bc6a064..4488b9b0ff 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q24.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q24.native_iceberg_compat/extended.txt @@ -4,96 +4,92 @@ CometNativeColumnarToRow +- Filter : +- Subquery : +- HashAggregate - : +- CometNativeColumnarToRow - : +- CometColumnarExchange + : +- Exchange + : +- HashAggregate : +- HashAggregate - : +- HashAggregate - : +- CometNativeColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin [COMET: Comet is not compatible with Spark for case conversion in locale-specific cases. Set spark.comet.caseConversion.enabled=true to enable it anyway.] - : :- CometNativeColumnarToRow - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometSortMergeJoin - : : : : : :- CometSort - : : : : : : +- CometExchange - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : +- CometSort - : : : : : +- CometExchange - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer - : +- BroadcastExchange - : +- CometNativeColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address + : +- Exchange + : +- HashAggregate + : +- Project + : +- BroadcastHashJoin [COMET: Comet is not compatible with Spark for case conversion in locale-specific cases. Set spark.comet.caseConversion.enabled=true to enable it anyway.] + : :- CometNativeColumnarToRow + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometProject + : : : : : +- CometSortMergeJoin + : : : : : :- CometSort + : : : : : : +- CometExchange + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales + : : : : : +- CometSort + : : : : : +- CometExchange + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer + : +- BroadcastExchange + : +- CometNativeColumnarToRow + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address +- HashAggregate - +- CometNativeColumnarToRow - +- CometColumnarExchange + +- Exchange + +- HashAggregate +- HashAggregate - +- HashAggregate - +- CometNativeColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Project - +- BroadcastHashJoin [COMET: Comet is not compatible with Spark for case conversion in locale-specific cases. Set spark.comet.caseConversion.enabled=true to enable it anyway.] - :- CometNativeColumnarToRow - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometSortMergeJoin - : : : : :- CometSort - : : : : : +- CometExchange - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : +- CometSort - : : : : +- CometExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer - +- BroadcastExchange - +- CometNativeColumnarToRow - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address + +- Exchange + +- HashAggregate + +- Project + +- BroadcastHashJoin [COMET: Comet is not compatible with Spark for case conversion in locale-specific cases. Set spark.comet.caseConversion.enabled=true to enable it anyway.] + :- CometNativeColumnarToRow + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometSortMergeJoin + : : : : :- CometSort + : : : : : +- CometExchange + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales + : : : : +- CometSort + : : : : +- CometExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer + +- BroadcastExchange + +- CometNativeColumnarToRow + +- CometProject + +- CometFilter + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address -Comet accelerated 72 out of 88 eligible operators (81%). Final plan contains 9 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 68 out of 88 eligible operators (77%). 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/q27a.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q27a.native_datafusion/extended.txt index 4eec8b0953..793de3abde 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,113 @@ 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 + : +- Exchange + : +- 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 + : +- Exchange + : +- 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 + +- Exchange + +- 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 -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 38 out of 95 eligible operators (40%). Final plan contains 16 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..575c853e76 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 @@ -5,43 +5,42 @@ CometNativeColumnarToRow +- 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 + : +- Exchange + : +- 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 -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 17 out of 37 eligible operators (45%). 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/q35.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q35.native_datafusion/extended.txt index 40a8ef4a6e..c8a9740a02 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 @@ -1,67 +1,66 @@ TakeOrderedAndProject +- HashAggregate - +- CometNativeColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- 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 - : : : : +- 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 - : : : +- 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 - : : +- 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 + +- Exchange + +- HashAggregate + +- Project + +- BroadcastHashJoin + :- Project + : +- BroadcastHashJoin + : :- 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 + : : : : +- 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 + : : : +- 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 + : : +- 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 -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 20 out of 54 eligible operators (37%). 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/q35.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q35.native_iceberg_compat/extended.txt index c5125b1861..72960fc267 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q35.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q35.native_iceberg_compat/extended.txt @@ -1,63 +1,62 @@ TakeOrderedAndProject +- HashAggregate - +- CometNativeColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- Project - : : +- Filter - : : +- BroadcastHashJoin - : : :- BroadcastHashJoin [COMET: Unsupported join type ExistenceJoin(exists#1)] - : : : :- CometNativeColumnarToRow - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : +- ReusedSubquery - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : +- ReusedSubquery - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- CometNativeColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address - +- BroadcastExchange - +- CometNativeColumnarToRow - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics + +- Exchange + +- HashAggregate + +- Project + +- BroadcastHashJoin + :- Project + : +- BroadcastHashJoin + : :- Project + : : +- Filter + : : +- BroadcastHashJoin + : : :- BroadcastHashJoin [COMET: Unsupported join type ExistenceJoin(exists#1)] + : : : :- CometNativeColumnarToRow + : : : : +- CometBroadcastHashJoin + : : : : :- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometNativeColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : +- BroadcastExchange + : : : +- CometNativeColumnarToRow + : : : +- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales + : : : : +- ReusedSubquery + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- BroadcastExchange + : : +- CometNativeColumnarToRow + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales + : : : +- ReusedSubquery + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- BroadcastExchange + : +- CometNativeColumnarToRow + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address + +- BroadcastExchange + +- CometNativeColumnarToRow + +- CometProject + +- CometFilter + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics -Comet accelerated 35 out of 54 eligible operators (64%). Final plan contains 7 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 34 out of 54 eligible operators (62%). 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 9091536e96..07ef74ce3d 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,64 @@ 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 + +- Exchange + +- 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 -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 20 out of 52 eligible operators (38%). 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/q36a.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q36a.native_datafusion/extended.txt index 1fc86dc5a8..380076587c 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 @@ -9,114 +9,109 @@ TakeOrderedAndProject +- 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 + : +- Exchange + : +- 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 + : +- Exchange + : +- HashAggregate : +- 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 + : +- Exchange + : +- 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 + +- Exchange + +- HashAggregate +- 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 + +- Exchange + +- 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 -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 40 out of 99 eligible operators (40%). Final plan contains 16 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..e347d0f433 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 @@ -12,36 +12,35 @@ TakeOrderedAndProject : : +- 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 + : : +- Exchange + : : +- 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 : +- 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).] @@ -49,36 +48,35 @@ TakeOrderedAndProject : +- 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 + : +- Exchange + : +- 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 +- 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).] @@ -86,35 +84,34 @@ TakeOrderedAndProject +- 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 + +- Exchange + +- 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 -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 33 out of 97 eligible operators (34%). 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/q49.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q49.native_datafusion/extended.txt index 0090739d55..d19eb04caf 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 @@ -13,33 +13,32 @@ 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 + : +- Exchange + : +- 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 :- Project : +- Filter : +- Window @@ -49,28 +48,27 @@ 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 + : +- Exchange + : +- 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 +- Project +- Filter +- Window @@ -80,27 +78,26 @@ 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 + +- Exchange + +- 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 -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 30 out of 87 eligible operators (34%). 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/q51a.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q51a.native_datafusion/extended.txt index 92ee33eda6..7bccd55531 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 @@ -14,111 +14,105 @@ TakeOrderedAndProject : :- CometSort : : +- CometColumnarExchange : : +- HashAggregate - : : +- CometNativeColumnarToRow - : : +- CometColumnarExchange - : : +- HashAggregate - : : +- Project - : : +- BroadcastHashJoin - : : :- 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 - : : +- 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 + : : +- Exchange + : : +- HashAggregate + : : +- Project + : : +- BroadcastHashJoin + : : :- 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 + : : : +- Exchange + : : : +- 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 + : : +- 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 + : : +- Exchange + : : +- 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 : +- CometSort : +- CometColumnarExchange : +- HashAggregate - : +- CometNativeColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- 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 - : +- 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 + : +- Exchange + : +- HashAggregate + : +- Project + : +- BroadcastHashJoin + : :- 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 + : : +- Exchange + : : +- 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 + : +- 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 + : +- Exchange + : +- 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 +- 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).] @@ -131,110 +125,104 @@ TakeOrderedAndProject :- CometSort : +- CometColumnarExchange : +- HashAggregate - : +- CometNativeColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- 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 - : +- 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 + : +- Exchange + : +- HashAggregate + : +- Project + : +- BroadcastHashJoin + : :- 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 + : : +- Exchange + : : +- 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 + : +- 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 + : +- Exchange + : +- 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 +- CometSort +- CometColumnarExchange +- HashAggregate - +- CometNativeColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Project - +- BroadcastHashJoin - :- 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 - +- 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 + +- Exchange + +- HashAggregate + +- Project + +- BroadcastHashJoin + :- 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 + : +- Exchange + : +- 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 + +- 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 + +- Exchange + +- 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 -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 70 out of 196 eligible operators (35%). Final plan contains 30 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_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q51a.native_iceberg_compat/extended.txt index 6560795de1..190830204f 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q51a.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q51a.native_iceberg_compat/extended.txt @@ -14,99 +14,97 @@ TakeOrderedAndProject : :- CometSort : : +- CometColumnarExchange : : +- HashAggregate - : : +- CometNativeColumnarToRow - : : +- CometColumnarExchange - : : +- HashAggregate - : : +- Project - : : +- BroadcastHashJoin - : : :- 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 - : : : +- CometExchange - : : : +- CometHashAggregate - : : : +- CometExchange - : : : +- CometHashAggregate - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] 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 - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- Exchange + : : +- HashAggregate + : : +- Project + : : +- BroadcastHashJoin + : : :- 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 + : : : +- CometExchange + : : : +- CometHashAggregate + : : : +- CometExchange + : : : +- CometHashAggregate + : : : +- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometNativeColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] 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 + : : +- CometExchange + : : +- CometHashAggregate + : : +- CometExchange + : : +- CometHashAggregate + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometNativeColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim : +- CometSort : +- CometColumnarExchange : +- HashAggregate - : +- CometNativeColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- 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 - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : +- ReusedSubquery - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] 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 - : +- CometExchange - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : +- ReusedSubquery - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- Exchange + : +- HashAggregate + : +- Project + : +- BroadcastHashJoin + : :- 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 + : : +- CometExchange + : : +- CometHashAggregate + : : +- CometExchange + : : +- CometHashAggregate + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales + : : : +- ReusedSubquery + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] 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 + : +- CometExchange + : +- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales + : : +- ReusedSubquery + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] 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).] @@ -119,98 +117,96 @@ TakeOrderedAndProject :- CometSort : +- CometColumnarExchange : +- HashAggregate - : +- CometNativeColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- 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 - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] 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 - : +- CometExchange - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- Exchange + : +- HashAggregate + : +- Project + : +- BroadcastHashJoin + : :- 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 + : : +- CometExchange + : : +- CometHashAggregate + : : +- CometExchange + : : +- CometHashAggregate + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometNativeColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] 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 + : +- CometExchange + : +- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales + : : +- SubqueryBroadcast + : : +- BroadcastExchange + : : +- CometNativeColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim +- CometSort +- CometColumnarExchange +- HashAggregate - +- CometNativeColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Project - +- BroadcastHashJoin - :- 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 - : +- CometExchange - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : +- ReusedSubquery - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] 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 - +- CometExchange - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : +- ReusedSubquery - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + +- Exchange + +- HashAggregate + +- Project + +- BroadcastHashJoin + :- 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 + : +- CometExchange + : +- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales + : : +- ReusedSubquery + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] 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 + +- CometExchange + +- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometProject + +- CometBroadcastHashJoin + :- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales + : +- ReusedSubquery + +- CometBroadcastExchange + +- CometProject + +- CometFilter + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Comet accelerated 138 out of 196 eligible operators (70%). Final plan contains 18 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 134 out of 196 eligible operators (68%). 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 66c510aaa0..50dc55f674 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 @@ -12,36 +12,35 @@ TakeOrderedAndProject : : +- 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 + : : +- Exchange + : : +- 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 : +- 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).] @@ -49,36 +48,35 @@ TakeOrderedAndProject : +- 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 + : +- Exchange + : +- 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 +- 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).] @@ -86,35 +84,34 @@ TakeOrderedAndProject +- 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 + +- Exchange + +- 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 -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 33 out of 97 eligible operators (34%). 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/q5a.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q5a.native_datafusion/extended.txt index ec2eef025a..858a24872a 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 @@ -5,318 +5,304 @@ CometNativeColumnarToRow +- 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 - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.web_site + : +- Exchange + : +- HashAggregate + : +- Union + : :- HashAggregate + : : +- Exchange + : : +- 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 + : : +- Exchange + : : +- 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 + : +- Exchange + : +- 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 + : +- Exchange + : +- HashAggregate : +- HashAggregate - : +- HashAggregate - : +- CometNativeColumnarToRow - : +- CometColumnarExchange + : +- Exchange + : +- HashAggregate + : +- Union + : :- HashAggregate + : : +- Exchange + : : +- 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 + : : +- Exchange + : : +- 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 - : +- 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 + : +- Exchange : +- 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 + : +- 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 + +- Exchange + +- HashAggregate +- HashAggregate - +- HashAggregate - +- CometNativeColumnarToRow - +- CometColumnarExchange + +- Exchange + +- HashAggregate + +- Union + :- HashAggregate + : +- Exchange + : +- 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 + : +- Exchange + : +- 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 - +- 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 + +- Exchange +- 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 + +- 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 -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 75 out of 263 eligible operators (28%). Final plan contains 43 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..18cc2a5c96 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,67 @@ 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 + +- Exchange + +- 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 -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 38 out of 58 eligible operators (65%). 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/q67a.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q67a.native_datafusion/extended.txt index 84fdb89cb0..010b7eccea 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 @@ -6,342 +6,325 @@ TakeOrderedAndProject +- 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 + : +- Exchange + : +- 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 + : +- Exchange + : +- HashAggregate : +- 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 + : +- Exchange + : +- 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 + : +- Exchange + : +- HashAggregate : +- 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 + : +- Exchange + : +- 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 + : +- Exchange + : +- HashAggregate : +- 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 + : +- Exchange + : +- 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 + : +- Exchange + : +- HashAggregate : +- 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 + : +- Exchange + : +- 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 + : +- Exchange + : +- HashAggregate : +- 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 + : +- Exchange + : +- 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 + : +- Exchange + : +- HashAggregate : +- 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 + : +- Exchange + : +- 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 + : +- Exchange + : +- HashAggregate : +- 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 + : +- Exchange + : +- 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 + +- Exchange + +- HashAggregate +- 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 + +- Exchange + +- 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 -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 110 out of 282 eligible operators (39%). Final plan contains 46 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..b92882dc80 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 @@ -9,177 +9,169 @@ TakeOrderedAndProject +- HashAggregate +- Union :- 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 - : +- Project - : +- BroadcastHashJoin - : :- CometNativeColumnarToRow - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.store - : +- 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 + : +- Exchange + : +- 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 + : +- Project + : +- BroadcastHashJoin + : :- CometNativeColumnarToRow + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.store + : +- BroadcastExchange + : +- Project + : +- Filter + : +- Window + : +- Sort + : +- HashAggregate + : +- Exchange + : +- 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 :- HashAggregate - : +- CometNativeColumnarToRow - : +- CometColumnarExchange + : +- Exchange + : +- HashAggregate : +- 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 - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- Project - : +- BroadcastHashJoin - : :- CometNativeColumnarToRow - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.store - : +- BroadcastExchange - : +- Project - : +- Filter - : +- Window - : +- Sort + : +- Exchange + : +- 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 + : +- Project + : +- BroadcastHashJoin + : :- CometNativeColumnarToRow + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.store + : +- BroadcastExchange + : +- Project + : +- Filter + : +- Window + : +- Sort + : +- HashAggregate + : +- Exchange : +- 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 + : +- 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 +- HashAggregate - +- CometNativeColumnarToRow - +- CometColumnarExchange + +- Exchange + +- HashAggregate +- 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 - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.date_dim - +- BroadcastExchange - +- Project - +- BroadcastHashJoin - :- CometNativeColumnarToRow - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.store - +- BroadcastExchange - +- Project - +- Filter - +- Window - +- Sort + +- Exchange + +- 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 + +- Project + +- BroadcastHashJoin + :- CometNativeColumnarToRow + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.store + +- BroadcastExchange + +- Project + +- Filter + +- Window + +- Sort + +- HashAggregate + +- Exchange +- 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 + +- 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 -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 46 out of 153 eligible operators (30%). 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/q70a.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q70a.native_iceberg_compat/extended.txt index a75ecb0e47..a9c5d2e774 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q70a.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q70a.native_iceberg_compat/extended.txt @@ -9,165 +9,160 @@ TakeOrderedAndProject +- HashAggregate +- Union :- HashAggregate - : +- CometNativeColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- CometNativeColumnarToRow - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- Project - : +- BroadcastHashJoin - : :- CometNativeColumnarToRow - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - : +- BroadcastExchange - : +- Project - : +- Filter - : +- 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 - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : +- ReusedSubquery - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- Exchange + : +- HashAggregate + : +- Project + : +- BroadcastHashJoin + : :- CometNativeColumnarToRow + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometNativeColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- BroadcastExchange + : +- Project + : +- BroadcastHashJoin + : :- CometNativeColumnarToRow + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store + : +- BroadcastExchange + : +- Project + : +- Filter + : +- 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 + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales + : : : +- ReusedSubquery + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim :- HashAggregate - : +- CometNativeColumnarToRow - : +- CometColumnarExchange + : +- Exchange + : +- HashAggregate : +- HashAggregate - : +- HashAggregate - : +- CometNativeColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- CometNativeColumnarToRow - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- Project - : +- BroadcastHashJoin - : :- CometNativeColumnarToRow - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - : +- BroadcastExchange - : +- Project - : +- Filter - : +- 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 + : +- Exchange + : +- HashAggregate + : +- Project + : +- BroadcastHashJoin + : :- CometNativeColumnarToRow + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometNativeColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- BroadcastExchange + : +- Project + : +- BroadcastHashJoin + : :- CometNativeColumnarToRow + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store + : +- BroadcastExchange + : +- Project + : +- Filter + : +- 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 - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : +- ReusedSubquery - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales + : : : +- ReusedSubquery + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim +- HashAggregate - +- CometNativeColumnarToRow - +- CometColumnarExchange + +- Exchange + +- HashAggregate +- HashAggregate - +- HashAggregate - +- CometNativeColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Project - +- BroadcastHashJoin - :- CometNativeColumnarToRow - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - +- BroadcastExchange - +- Project - +- BroadcastHashJoin - :- CometNativeColumnarToRow - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - +- BroadcastExchange - +- Project - +- Filter - +- 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 + +- Exchange + +- HashAggregate + +- Project + +- BroadcastHashJoin + :- CometNativeColumnarToRow + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales + : : +- SubqueryBroadcast + : : +- BroadcastExchange + : : +- CometNativeColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + +- BroadcastExchange + +- Project + +- BroadcastHashJoin + :- CometNativeColumnarToRow + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store + +- BroadcastExchange + +- Project + +- Filter + +- 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 - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : +- ReusedSubquery - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales + : : +- ReusedSubquery + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store + +- CometBroadcastExchange + +- CometProject + +- CometFilter + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Comet accelerated 102 out of 153 eligible operators (66%). Final plan contains 18 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 97 out of 153 eligible operators (63%). 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-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..00d2b3b0d7 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 @@ -6,100 +6,96 @@ TakeOrderedAndProject : :- 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 + : : : +- Exchange + : : : +- 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 + : : +- Exchange + : : +- 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 + : +- Exchange + : +- 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 + +- Exchange + +- 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 -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 24 out of 85 eligible operators (28%). 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/q77a.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q77a.native_datafusion/extended.txt index 178b946ecb..d11e87a30b 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 @@ -5,405 +5,382 @@ CometNativeColumnarToRow +- HashAggregate +- Union :- HashAggregate - : +- CometNativeColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Union - : :- 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 - : : :- 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 + : +- Exchange + : +- HashAggregate + : +- Union + : :- Project + : : +- BroadcastHashJoin + : : :- HashAggregate + : : : +- Exchange + : : : +- 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 + : : +- Exchange + : : +- 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 + : : :- BroadcastExchange + : : : +- HashAggregate + : : : +- Exchange + : : : +- 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 + : : +- Exchange + : : +- 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 + : : +- Exchange + : : +- 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 + : +- Exchange + : +- 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 :- HashAggregate - : +- CometNativeColumnarToRow - : +- CometColumnarExchange + : +- Exchange + : +- HashAggregate : +- HashAggregate - : +- HashAggregate - : +- CometNativeColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Union - : :- 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 - : : :- 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 + : +- Exchange + : +- HashAggregate + : +- Union + : :- Project + : : +- BroadcastHashJoin + : : :- HashAggregate + : : : +- Exchange + : : : +- 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 + : : +- Exchange + : : +- 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 + : : :- BroadcastExchange + : : : +- HashAggregate + : : : +- Exchange + : : : +- 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 + : : +- Exchange + : : +- 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 + : : +- Exchange : : +- 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 + : : +- 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 + : +- Exchange : +- 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 + : +- 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 +- HashAggregate - +- CometNativeColumnarToRow - +- CometColumnarExchange + +- Exchange + +- HashAggregate +- HashAggregate - +- HashAggregate - +- CometNativeColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Union - :- 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 - : :- 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 + +- Exchange + +- HashAggregate + +- Union + :- Project + : +- BroadcastHashJoin + : :- HashAggregate + : : +- Exchange + : : +- 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 + : +- Exchange + : +- 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 + : :- BroadcastExchange + : : +- HashAggregate + : : +- Exchange + : : +- 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 + : +- Exchange + : +- 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 + : +- Exchange : +- 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 + : +- 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 + +- Exchange +- 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 + +- 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 -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 90 out of 332 eligible operators (27%). Final plan contains 52 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_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q77a.native_iceberg_compat/extended.txt index 3578f807aa..82e3d521e5 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q77a.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q77a.native_iceberg_compat/extended.txt @@ -5,351 +5,346 @@ CometNativeColumnarToRow +- HashAggregate +- Union :- HashAggregate - : +- CometNativeColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Union - : :- CometNativeColumnarToRow - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometHashAggregate - : : : +- CometExchange - : : : +- CometHashAggregate - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - : : +- CometBroadcastExchange - : : +- CometHashAggregate - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns - : : : : +- ReusedSubquery - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - : :- Project - : : +- BroadcastNestedLoopJoin [COMET: BroadcastNestedLoopJoin is not supported] - : : :- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometHashAggregate - : : : +- CometExchange - : : : +- CometHashAggregate - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : +- ReusedSubquery - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometNativeColumnarToRow - : : +- CometHashAggregate - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns - : : : +- ReusedSubquery - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometNativeColumnarToRow - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometHashAggregate - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : +- ReusedSubquery - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_page - : +- CometBroadcastExchange - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns - : : : +- ReusedSubquery - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_page + : +- Exchange + : +- HashAggregate + : +- Union + : :- CometNativeColumnarToRow + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometHashAggregate + : : : +- CometExchange + : : : +- CometHashAggregate + : : : +- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometNativeColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store + : : +- CometBroadcastExchange + : : +- CometHashAggregate + : : +- CometExchange + : : +- CometHashAggregate + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns + : : : : +- ReusedSubquery + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store + : :- Project + : : +- BroadcastNestedLoopJoin [COMET: BroadcastNestedLoopJoin is not supported] + : : :- BroadcastExchange + : : : +- CometNativeColumnarToRow + : : : +- CometHashAggregate + : : : +- CometExchange + : : : +- CometHashAggregate + : : : +- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales + : : : : +- ReusedSubquery + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- CometNativeColumnarToRow + : : +- CometHashAggregate + : : +- CometExchange + : : +- CometHashAggregate + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns + : : : +- ReusedSubquery + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- CometNativeColumnarToRow + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometHashAggregate + : : +- CometExchange + : : +- CometHashAggregate + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales + : : : : +- ReusedSubquery + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_page + : +- CometBroadcastExchange + : +- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns + : : : +- ReusedSubquery + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_page :- HashAggregate - : +- CometNativeColumnarToRow - : +- CometColumnarExchange + : +- Exchange + : +- HashAggregate : +- HashAggregate - : +- HashAggregate - : +- CometNativeColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Union - : :- CometNativeColumnarToRow - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometHashAggregate - : : : +- CometExchange - : : : +- CometHashAggregate - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - : : +- CometBroadcastExchange - : : +- CometHashAggregate - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns - : : : : +- ReusedSubquery - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - : :- Project - : : +- BroadcastNestedLoopJoin [COMET: BroadcastNestedLoopJoin is not supported] - : : :- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometHashAggregate - : : : +- CometExchange - : : : +- CometHashAggregate - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : +- ReusedSubquery - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometNativeColumnarToRow - : : +- CometHashAggregate - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns - : : : +- ReusedSubquery - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometNativeColumnarToRow - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometHashAggregate - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : +- ReusedSubquery - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_page - : +- CometBroadcastExchange + : +- Exchange + : +- HashAggregate + : +- Union + : :- CometNativeColumnarToRow + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometHashAggregate + : : : +- CometExchange + : : : +- CometHashAggregate + : : : +- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- CometNativeColumnarToRow + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store + : : +- CometBroadcastExchange + : : +- CometHashAggregate + : : +- CometExchange + : : +- CometHashAggregate + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns + : : : : +- ReusedSubquery + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store + : :- Project + : : +- BroadcastNestedLoopJoin [COMET: BroadcastNestedLoopJoin is not supported] + : : :- BroadcastExchange + : : : +- CometNativeColumnarToRow + : : : +- CometHashAggregate + : : : +- CometExchange + : : : +- CometHashAggregate + : : : +- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales + : : : : +- ReusedSubquery + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- CometNativeColumnarToRow + : : +- CometHashAggregate + : : +- CometExchange + : : +- CometHashAggregate + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns + : : : +- ReusedSubquery + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- CometNativeColumnarToRow + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometHashAggregate + : : +- CometExchange + : : +- CometHashAggregate + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales + : : : : +- ReusedSubquery + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_page + : +- CometBroadcastExchange + : +- CometHashAggregate + : +- CometExchange : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns - : : : +- ReusedSubquery - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_page + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns + : : : +- ReusedSubquery + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_page +- HashAggregate - +- CometNativeColumnarToRow - +- CometColumnarExchange + +- Exchange + +- HashAggregate +- HashAggregate - +- HashAggregate - +- CometNativeColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Union - :- CometNativeColumnarToRow - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometHashAggregate - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - : +- CometBroadcastExchange - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns - : : : +- ReusedSubquery - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - :- Project - : +- BroadcastNestedLoopJoin [COMET: BroadcastNestedLoopJoin is not supported] - : :- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometHashAggregate - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : +- ReusedSubquery - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometNativeColumnarToRow - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns - : : +- ReusedSubquery - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - +- CometNativeColumnarToRow - +- CometProject - +- CometBroadcastHashJoin - :- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : +- ReusedSubquery - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_page - +- CometBroadcastExchange + +- Exchange + +- HashAggregate + +- Union + :- CometNativeColumnarToRow + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometHashAggregate + : : +- CometExchange + : : +- CometHashAggregate + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometNativeColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store + : +- CometBroadcastExchange + : +- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns + : : : +- ReusedSubquery + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store + :- Project + : +- BroadcastNestedLoopJoin [COMET: BroadcastNestedLoopJoin is not supported] + : :- BroadcastExchange + : : +- CometNativeColumnarToRow + : : +- CometHashAggregate + : : +- CometExchange + : : +- CometHashAggregate + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales + : : : +- ReusedSubquery + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- CometNativeColumnarToRow + : +- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns + : : +- ReusedSubquery + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + +- CometNativeColumnarToRow + +- CometProject + +- CometBroadcastHashJoin + :- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales + : : : +- ReusedSubquery + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_page + +- CometBroadcastExchange + +- CometHashAggregate + +- CometExchange +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns - : : +- ReusedSubquery - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - +- CometBroadcastExchange - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_page + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns + : : +- ReusedSubquery + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + +- CometBroadcastExchange + +- CometFilter + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_page -Comet accelerated 287 out of 332 eligible operators (86%). Final plan contains 21 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 282 out of 332 eligible operators (84%). Final plan contains 16 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..bdd904b098 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 @@ -9,93 +9,88 @@ TakeOrderedAndProject +- 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 + : +- Exchange + : +- 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 + : +- Exchange + : +- HashAggregate : +- 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 + : +- Exchange + : +- 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 + +- Exchange + +- HashAggregate +- 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 + +- Exchange + +- 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 -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 31 out of 81 eligible operators (38%). 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-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..4d3c625e71 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 @@ -7,31 +7,30 @@ 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 + +- Exchange + +- 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 -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 13 out of 28 eligible operators (46%). Final plan contains 6 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/scala/org/apache/comet/rules/CometExecRuleSuite.scala b/spark/src/test/scala/org/apache/comet/rules/CometExecRuleSuite.scala index cf6f8918f4..9d9b24153f 100644 --- a/spark/src/test/scala/org/apache/comet/rules/CometExecRuleSuite.scala +++ b/spark/src/test/scala/org/apache/comet/rules/CometExecRuleSuite.scala @@ -229,4 +229,37 @@ class CometExecRuleSuite extends CometTestBase { } } + test("CometExecRule should not wrap shuffle in CometColumnarShuffle when both sides are JVM") { + withTempView("test_data") { + createTestDataFrame.createOrReplaceTempView("test_data") + + val sparkPlan = + createSparkPlan(spark, "SELECT COUNT(*), SUM(id) FROM test_data GROUP BY (id % 3)") + + val originalShuffleExchangeCount = countOperators(sparkPlan, classOf[ShuffleExchangeExec]) + assert(originalShuffleExchangeCount == 1) + assert(countOperators(sparkPlan, classOf[HashAggregateExec]) == 2) + + // Disable partial aggregate so both aggregates fall back to Spark JVM. The shuffle between + // them would otherwise be wrapped with CometColumnarShuffle, which adds unnecessary + // row<->arrow conversion overhead when neither side can consume columnar output. + // See https://github.com/apache/datafusion-comet/issues/4004. + withSQLConf( + CometConf.COMET_ENABLE_PARTIAL_HASH_AGGREGATE.key -> "false", + CometConf.COMET_EXEC_LOCAL_TABLE_SCAN_ENABLED.key -> "true") { + val transformedPlan = applyCometExecRule(sparkPlan) + + // Both aggregates should remain JVM + assert(countOperators(transformedPlan, classOf[HashAggregateExec]) == 2) + assert(countOperators(transformedPlan, classOf[CometHashAggregateExec]) == 0) + + // The shuffle should remain a Spark ShuffleExchangeExec (not wrapped in Comet) + assert(countOperators(transformedPlan, classOf[CometShuffleExchangeExec]) == 0) + assert( + countOperators(transformedPlan, classOf[ShuffleExchangeExec]) == + originalShuffleExchangeCount) + } + } + } + }