diff --git a/.gitignore b/.gitignore index a3c97ff992..33eaaa82dd 100644 --- a/.gitignore +++ b/.gitignore @@ -24,6 +24,7 @@ spark/benchmarks comet-event-trace.json __pycache__ output +.claude/ docs/comet-*/ docs/build/ docs/temp/ diff --git a/dev/diffs/3.4.3.diff b/dev/diffs/3.4.3.diff index 8a939f2028..ed856c1592 100644 --- a/dev/diffs/3.4.3.diff +++ b/dev/diffs/3.4.3.diff @@ -500,7 +500,17 @@ diff --git a/sql/core/src/test/scala/org/apache/spark/sql/ExplainSuite.scala b/s index a6b295578d6..91acca4306f 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/ExplainSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/ExplainSuite.scala -@@ -463,7 +463,8 @@ class ExplainSuite extends ExplainSuiteHelper with DisableAdaptiveExecutionSuite +@@ -260,7 +260,8 @@ class ExplainSuite extends ExplainSuiteHelper with DisableAdaptiveExecutionSuite + } + } + +- test("SPARK-33853: explain codegen - check presence of subquery") { ++ test("SPARK-33853: explain codegen - check presence of subquery", ++ IgnoreComet("Comet changes the WholeStageCodegen subtree count")) { + withSQLConf(SQLConf.WHOLESTAGE_CODEGEN_ENABLED.key -> "true") { + withTempView("df") { + val df1 = spark.range(1, 100) +@@ -463,7 +464,8 @@ class ExplainSuite extends ExplainSuiteHelper with DisableAdaptiveExecutionSuite } } @@ -510,7 +520,7 @@ index a6b295578d6..91acca4306f 100644 withTempDir { dir => Seq("parquet", "orc", "csv", "json").foreach { fmt => val basePath = dir.getCanonicalPath + "/" + fmt -@@ -541,7 +542,9 @@ class ExplainSuite extends ExplainSuiteHelper with DisableAdaptiveExecutionSuite +@@ -541,7 +543,9 @@ class ExplainSuite extends ExplainSuiteHelper with DisableAdaptiveExecutionSuite } } diff --git a/dev/diffs/3.5.8.diff b/dev/diffs/3.5.8.diff index d3a5c617dc..3789cebb10 100644 --- a/dev/diffs/3.5.8.diff +++ b/dev/diffs/3.5.8.diff @@ -478,10 +478,20 @@ index f33432ddb6f..b375e285dde 100644 } assert(scanOption.isDefined) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/ExplainSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/ExplainSuite.scala -index a206e97c353..fea1149b67d 100644 +index a206e97c353..8bd3ab5985a 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/ExplainSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/ExplainSuite.scala -@@ -467,7 +467,8 @@ class ExplainSuite extends ExplainSuiteHelper with DisableAdaptiveExecutionSuite +@@ -264,7 +264,8 @@ class ExplainSuite extends ExplainSuiteHelper with DisableAdaptiveExecutionSuite + } + } + +- test("SPARK-33853: explain codegen - check presence of subquery") { ++ test("SPARK-33853: explain codegen - check presence of subquery", ++ IgnoreComet("Comet changes the WholeStageCodegen subtree count")) { + withSQLConf(SQLConf.WHOLESTAGE_CODEGEN_ENABLED.key -> "true") { + withTempView("df") { + val df1 = spark.range(1, 100) +@@ -467,7 +468,8 @@ class ExplainSuite extends ExplainSuiteHelper with DisableAdaptiveExecutionSuite } } @@ -491,7 +501,7 @@ index a206e97c353..fea1149b67d 100644 withTempDir { dir => Seq("parquet", "orc", "csv", "json").foreach { fmt => val basePath = dir.getCanonicalPath + "/" + fmt -@@ -545,7 +546,9 @@ class ExplainSuite extends ExplainSuiteHelper with DisableAdaptiveExecutionSuite +@@ -545,7 +547,9 @@ class ExplainSuite extends ExplainSuiteHelper with DisableAdaptiveExecutionSuite } } diff --git a/dev/diffs/4.0.2.diff b/dev/diffs/4.0.2.diff index 436491f9c8..3fcce17ad1 100644 --- a/dev/diffs/4.0.2.diff +++ b/dev/diffs/4.0.2.diff @@ -615,10 +615,20 @@ index 2c24cc7d570..12d897866da 100644 } assert(scanOption.isDefined) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/ExplainSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/ExplainSuite.scala -index 9c90e0105a4..fadf2f0f698 100644 +index 9c90e0105a4..ed6d4887b13 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/ExplainSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/ExplainSuite.scala -@@ -470,7 +470,8 @@ class ExplainSuite extends ExplainSuiteHelper with DisableAdaptiveExecutionSuite +@@ -267,7 +267,8 @@ class ExplainSuite extends ExplainSuiteHelper with DisableAdaptiveExecutionSuite + } + } + +- test("SPARK-33853: explain codegen - check presence of subquery") { ++ test("SPARK-33853: explain codegen - check presence of subquery", ++ IgnoreComet("Comet changes the WholeStageCodegen subtree count")) { + withSQLConf(SQLConf.WHOLESTAGE_CODEGEN_ENABLED.key -> "true") { + withTempView("df") { + val df1 = spark.range(1, 100) +@@ -470,7 +471,8 @@ class ExplainSuite extends ExplainSuiteHelper with DisableAdaptiveExecutionSuite } } @@ -628,7 +638,7 @@ index 9c90e0105a4..fadf2f0f698 100644 withTempDir { dir => Seq("parquet", "orc", "csv", "json").foreach { fmt => val basePath = dir.getCanonicalPath + "/" + fmt -@@ -548,7 +549,9 @@ class ExplainSuite extends ExplainSuiteHelper with DisableAdaptiveExecutionSuite +@@ -548,7 +550,9 @@ class ExplainSuite extends ExplainSuiteHelper with DisableAdaptiveExecutionSuite } } 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 06d926dfc0..e5453bee57 100644 --- a/spark/src/main/scala/org/apache/comet/rules/CometExecRule.scala +++ b/spark/src/main/scala/org/apache/comet/rules/CometExecRule.scala @@ -23,15 +23,17 @@ import scala.collection.mutable.ListBuffer import org.apache.spark.sql.SparkSession import org.apache.spark.sql.catalyst.expressions.{Divide, DoubleLiteral, EqualNullSafe, EqualTo, Expression, FloatLiteral, GreaterThan, GreaterThanOrEqual, KnownFloatingPointNormalized, LessThan, LessThanOrEqual, NamedExpression, Remainder} +import org.apache.spark.sql.catalyst.expressions.aggregate.{AggregateMode, Final, Partial} import org.apache.spark.sql.catalyst.optimizer.NormalizeNaNAndZero import org.apache.spark.sql.catalyst.rules.Rule +import org.apache.spark.sql.catalyst.trees.TreeNodeTag import org.apache.spark.sql.catalyst.util.sideBySide import org.apache.spark.sql.comet._ import org.apache.spark.sql.comet.execution.shuffle.{CometColumnarShuffle, CometNativeShuffle, CometShuffleExchangeExec} import org.apache.spark.sql.comet.util.Utils import org.apache.spark.sql.execution._ import org.apache.spark.sql.execution.adaptive.{AdaptiveSparkPlanExec, AQEShuffleReadExec, BroadcastQueryStageExec, ShuffleQueryStageExec} -import org.apache.spark.sql.execution.aggregate.{HashAggregateExec, ObjectHashAggregateExec} +import org.apache.spark.sql.execution.aggregate.{BaseAggregateExec, HashAggregateExec, ObjectHashAggregateExec} import org.apache.spark.sql.execution.command.{DataWritingCommandExec, ExecutedCommandExec} import org.apache.spark.sql.execution.datasources.WriteFilesExec import org.apache.spark.sql.execution.datasources.csv.CSVFileFormat @@ -57,6 +59,14 @@ import org.apache.comet.shims.{ShimCometStreaming, ShimSubqueryBroadcast} object CometExecRule { + /** + * Tag applied to Partial-mode aggregate operators that must NOT be converted to Comet because + * the corresponding Final-mode aggregate cannot be converted, and the aggregate functions have + * incompatible intermediate buffer formats between Spark and Comet. + */ + val COMET_UNSAFE_PARTIAL: TreeNodeTag[String] = + TreeNodeTag[String]("comet.unsafePartialAgg") + /** * Fully native operators. */ @@ -558,6 +568,12 @@ case class CometExecRule(session: SparkSession) normalizedPlan } + // Tag Partial aggregates that must not be converted to Comet because the + // corresponding Final aggregate cannot be converted and the intermediate buffer + // formats are incompatible. This runs before transform() so the tags are checked + // during the bottom-up conversion. Tags persist through AQE stage creation. + tagUnsafePartialAggregates(planWithJoinRewritten) + var newPlan = transform(planWithJoinRewritten) // if the plan cannot be run fully natively then explain why (when appropriate @@ -778,4 +794,127 @@ case class CometExecRule(session: SparkSession) } } + /** + * Walk the plan to find Final-mode aggregates that cannot be converted to Comet. For each such + * Final, if the aggregate functions have incompatible intermediate buffer formats, tag the + * corresponding Partial-mode aggregate so it will also be skipped during conversion. + * + * This prevents the crash described in issue #1389 where a Comet Partial produces intermediate + * data in a format that the Spark Final cannot interpret. + */ + private def tagUnsafePartialAggregates(plan: SparkPlan): Unit = { + plan.foreach { + case agg: BaseAggregateExec => + // Only consider single-mode Final aggregates. Multi-mode Finals come from Spark's + // distinct-aggregate rewrite, where the Comet partial (if any) feeds into a Spark + // PartialMerge rather than directly into a Final, which is a different code path + // than the Comet-Partial → Spark-Final crash scenario from issue #1389. + val modes = agg.aggregateExpressions.map(_.mode).distinct + if (modes == Seq(Final) && + !QueryPlanSerde.allAggsSupportMixedExecution(agg.aggregateExpressions) && + !canAggregateBeConverted(agg, Final)) { + findPartialAggInPlan(agg.child).foreach { partial => + // Only tag if the Partial would otherwise have been converted. If the Partial + // itself cannot be converted (e.g. the aggregate function is incompatible for the + // input type), there is no buffer-format mismatch to guard against, and tagging + // would mask the natural, more specific fallback reason. + if (canAggregateBeConverted(partial, Partial)) { + partial.setTagValue( + CometExecRule.COMET_UNSAFE_PARTIAL, + "Partial aggregate disabled: corresponding final aggregate " + + "cannot be converted to Comet and intermediate buffer formats are incompatible") + } + } + } + case _ => + } + } + + /** + * Conservative check for whether an aggregate could be converted to Comet. Checks operator + * enablement, grouping expressions, aggregate expressions, and result expressions. + * Intentionally skips the sparkFinalMode / child-native checks since those depend on + * transformation state. + * + * WARNING: this intentionally mirrors the predicate checks in `CometBaseAggregate.doConvert` + * (operators.scala). Any change to the convertibility rules there must be reflected here or + * this tagging pass will drift and either crash (missed tag) or over-disable (spurious tag). A + * shared predicate helper would be preferable. + */ + private def canAggregateBeConverted( + agg: BaseAggregateExec, + expectedMode: AggregateMode): Boolean = { + val handler = allExecs.get(agg.getClass) + if (handler.isEmpty) return false + val serde = handler.get.asInstanceOf[CometOperatorSerde[SparkPlan]] + if (!isOperatorEnabled(serde, agg.asInstanceOf[SparkPlan])) return false + + // ObjectHashAggregate has an extra shuffle-enabled guard in its convert method + agg match { + case _: ObjectHashAggregateExec if !isCometShuffleEnabled(agg.conf) => return false + case _ => + } + + val aggregateExpressions = agg.aggregateExpressions + val groupingExpressions = agg.groupingExpressions + + if (groupingExpressions.isEmpty && aggregateExpressions.isEmpty) return false + + if (groupingExpressions.exists(e => QueryPlanSerde.containsMapType(e.dataType))) return false + + if (!groupingExpressions.forall(e => + QueryPlanSerde.exprToProto(e, agg.child.output).isDefined)) { + return false + } + + if (aggregateExpressions.isEmpty) { + // Result expressions always checked when there are no aggregate expressions + val attributes = + groupingExpressions.map(_.toAttribute) ++ agg.aggregateAttributes + return agg.resultExpressions.forall(e => + QueryPlanSerde.exprToProto(e, attributes).isDefined) + } + + val modes = aggregateExpressions.map(_.mode).distinct + if (modes.size != 1 || modes.head != expectedMode) return false + + // In Final mode, exprToProto resolves against the child's output; in Partial/non-Final mode + // it must bind to input attributes. This mirrors the `binding` calculation in + // `CometBaseAggregate.doConvert`. + val binding = expectedMode != Final + if (!aggregateExpressions.forall(e => + QueryPlanSerde.aggExprToProto(e, agg.child.output, binding, agg.conf).isDefined)) { + return false + } + + // doConvert only checks resultExpressions in Final mode when aggregate expressions exist + // (Partial emits the buffer directly). Mirror that here to avoid false negatives. + if (expectedMode == Final) { + val attributes = + groupingExpressions.map(_.toAttribute) ++ agg.aggregateAttributes + agg.resultExpressions.forall(e => QueryPlanSerde.exprToProto(e, attributes).isDefined) + } else { + true + } + } + + /** + * Look for a Partial-mode aggregate that feeds directly into the given plan (the child of a + * Final). Walks through exchanges and AQE stages only, stopping at anything else including + * other aggregate stages. This avoids tagging unrelated Partials found deeper in the plan (e.g. + * the non-distinct Partial in a distinct-aggregate rewrite, which is separated from the Final + * by intermediate PartialMerge stages). Requires `aggregateExpressions.nonEmpty` so that + * group-by-only dedup stages are not mistaken for the partial we want to tag. + */ + private def findPartialAggInPlan(plan: SparkPlan): Option[BaseAggregateExec] = plan match { + case agg: BaseAggregateExec + if agg.aggregateExpressions.nonEmpty && + agg.aggregateExpressions.forall(e => e.mode == Partial) => + Some(agg) + case a: AQEShuffleReadExec => findPartialAggInPlan(a.child) + case s: ShuffleQueryStageExec => findPartialAggInPlan(s.plan) + case e: ShuffleExchangeExec => findPartialAggInPlan(e.child) + case _ => None + } + } diff --git a/spark/src/main/scala/org/apache/comet/serde/CometAggregateExpressionSerde.scala b/spark/src/main/scala/org/apache/comet/serde/CometAggregateExpressionSerde.scala index 316510400b..9a83152168 100644 --- a/spark/src/main/scala/org/apache/comet/serde/CometAggregateExpressionSerde.scala +++ b/spark/src/main/scala/org/apache/comet/serde/CometAggregateExpressionSerde.scala @@ -81,6 +81,17 @@ trait CometAggregateExpressionSerde[T <: AggregateFunction] { */ def getSupportLevel(expr: T): SupportLevel = Compatible(None) + /** + * Whether this aggregate's intermediate buffer format is compatible between Spark and Comet, + * making it safe to run the Partial in one engine and the Final in the other. Aggregates with + * simple single-value buffers (MIN, MAX, bitwise) are safe; those with complex or + * differently-encoded buffers (AVG, SUM with decimals, CollectSet, Variance) are not. COUNT is + * intentionally excluded: mixed COUNT partial/final regressed AQE's + * PropagateEmptyRelationAfterAQE pattern (which matches BaseAggregateExec only) and the Spark + * 4.0 count-bug decorrelation for correlated IN subqueries. + */ + def supportsMixedPartialFinal: Boolean = false + /** * Convert a Spark expression into a protocol buffer representation that can be passed into * native code. diff --git a/spark/src/main/scala/org/apache/comet/serde/QueryPlanSerde.scala b/spark/src/main/scala/org/apache/comet/serde/QueryPlanSerde.scala index 3fc4adb623..7867692be2 100644 --- a/spark/src/main/scala/org/apache/comet/serde/QueryPlanSerde.scala +++ b/spark/src/main/scala/org/apache/comet/serde/QueryPlanSerde.scala @@ -291,6 +291,24 @@ object QueryPlanSerde extends Logging with CometExprShim with CometTypeShim { classOf[VariancePop] -> CometVariancePop, classOf[VarianceSamp] -> CometVarianceSamp) + /** + * Returns true if all aggregate expressions in the list have intermediate buffer formats that + * are compatible between Spark and Comet, making it safe to run Partial in one engine and Final + * in the other. + */ + def allAggsSupportMixedExecution(aggExprs: Seq[AggregateExpression]): Boolean = { + aggExprs.forall { aggExpr => + val fn = aggExpr.aggregateFunction + aggrSerdeMap.get(fn.getClass) match { + case Some(handler) => + handler + .asInstanceOf[CometAggregateExpressionSerde[AggregateFunction]] + .supportsMixedPartialFinal + case None => false + } + } + } + // A unique id for each expression. ~used to look up QueryContext during error creation. private val exprIdCounter = new AtomicLong(0) @@ -368,6 +386,19 @@ object QueryPlanSerde extends Logging with CometExprShim with CometTypeShim { false } + /** + * Returns true if the given data type is or contains a `MapType` at any nesting level. Arrow's + * row format (used by DataFusion's grouped hash aggregate for composite group keys) does not + * support `Map`, so grouping on any type that transitively contains a map would crash in native + * execution. + */ + def containsMapType(dt: DataType): Boolean = dt match { + case _: MapType => true + case a: ArrayType => containsMapType(a.elementType) + case s: StructType => s.fields.exists(f => containsMapType(f.dataType)) + case _ => false + } + /** * Serializes Spark datatype to protobuf. Note that, a datatype can be serialized by this method * doesn't mean it is supported by Comet native execution, i.e., `supportedDataType` may return diff --git a/spark/src/main/scala/org/apache/comet/serde/aggregates.scala b/spark/src/main/scala/org/apache/comet/serde/aggregates.scala index bb0466b4eb..e7227ccca9 100644 --- a/spark/src/main/scala/org/apache/comet/serde/aggregates.scala +++ b/spark/src/main/scala/org/apache/comet/serde/aggregates.scala @@ -34,6 +34,8 @@ import org.apache.comet.shims.CometEvalModeUtil object CometMin extends CometAggregateExpressionSerde[Min] { + override def supportsMixedPartialFinal: Boolean = true + override def convert( aggExpr: AggregateExpression, expr: Min, @@ -81,6 +83,8 @@ object CometMin extends CometAggregateExpressionSerde[Min] { object CometMax extends CometAggregateExpressionSerde[Max] { + override def supportsMixedPartialFinal: Boolean = true + override def convert( aggExpr: AggregateExpression, expr: Max, @@ -319,6 +323,8 @@ object CometLast extends CometAggregateExpressionSerde[Last] { } object CometBitAndAgg extends CometAggregateExpressionSerde[BitAndAgg] { + override def supportsMixedPartialFinal: Boolean = true + override def convert( aggExpr: AggregateExpression, bitAnd: BitAndAgg, @@ -353,6 +359,8 @@ object CometBitAndAgg extends CometAggregateExpressionSerde[BitAndAgg] { } object CometBitOrAgg extends CometAggregateExpressionSerde[BitOrAgg] { + override def supportsMixedPartialFinal: Boolean = true + override def convert( aggExpr: AggregateExpression, bitOr: BitOrAgg, @@ -387,6 +395,8 @@ object CometBitOrAgg extends CometAggregateExpressionSerde[BitOrAgg] { } object CometBitXOrAgg extends CometAggregateExpressionSerde[BitXorAgg] { + override def supportsMixedPartialFinal: Boolean = true + override def convert( aggExpr: AggregateExpression, bitXor: BitXorAgg, diff --git a/spark/src/main/scala/org/apache/spark/sql/comet/operators.scala b/spark/src/main/scala/org/apache/spark/sql/comet/operators.scala index 888df13bac..38753a350f 100644 --- a/spark/src/main/scala/org/apache/spark/sql/comet/operators.scala +++ b/spark/src/main/scala/org/apache/spark/sql/comet/operators.scala @@ -54,8 +54,10 @@ import com.google.protobuf.CodedOutputStream import org.apache.comet.{CometConf, CometExecIterator, CometRuntimeException, ConfigEntry} import org.apache.comet.CometSparkSessionExtensions.{isCometShuffleEnabled, withInfo} import org.apache.comet.parquet.CometParquetUtils +import org.apache.comet.rules.CometExecRule import org.apache.comet.serde.{CometOperatorSerde, Compatible, Incompatible, OperatorOuterClass, SupportLevel, Unsupported} import org.apache.comet.serde.OperatorOuterClass.{AggregateMode => CometAggregateMode, Operator} +import org.apache.comet.serde.QueryPlanSerde import org.apache.comet.serde.QueryPlanSerde.{aggExprToProto, exprToProto, isStringCollationType, supportedSortType} import org.apache.comet.serde.operator.CometSink @@ -1362,10 +1364,24 @@ trait CometBaseAggregate { // We support {Partial, PartialMerge} mix; other combinations are rejected. val multiMode = modes.size > 1 && modeSet != Set(Partial, PartialMerge) // For a final mode HashAggregate, we only need to transform the HashAggregate - // if there is Comet partial aggregation. + // if there is Comet partial aggregation, unless all aggregates have compatible + // intermediate buffer formats (safe for mixed Spark/Comet execution). val sparkFinalMode = modes.contains(Final) && findCometPartialAgg(aggregate.child).isEmpty - if (multiMode || sparkFinalMode) { + if (multiMode) { + return None + } + + if (sparkFinalMode && + !QueryPlanSerde.allAggsSupportMixedExecution(aggregate.aggregateExpressions)) { + return None + } + + // Check if this aggregate has been tagged as unsafe for mixed execution + // (Comet partial + Spark final with incompatible intermediate buffers) + val unsafeReason = aggregate.getTagValue(CometExecRule.COMET_UNSAFE_PARTIAL) + if (unsafeReason.isDefined) { + withInfo(aggregate, unsafeReason.get) return None } @@ -1380,11 +1396,7 @@ trait CometBaseAggregate { return None } - if (groupingExpressions.exists(expr => - expr.dataType match { - case _: MapType => true - case _ => false - })) { + if (groupingExpressions.exists(expr => QueryPlanSerde.containsMapType(expr.dataType))) { withInfo(aggregate, "Grouping on map types is not supported") return None } diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_2/q1.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_2/q1.native_datafusion/extended.txt deleted file mode 100644 index f49109dafb..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_2/q1.native_datafusion/extended.txt +++ /dev/null @@ -1,52 +0,0 @@ -CometNativeColumnarToRow -+- CometTakeOrderedAndProject - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometHashAggregate - : : : +- CometExchange - : : : +- CometHashAggregate - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.store_returns - : : : : +- CometSubqueryBroadcast - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometFilter - : : +- CometHashAggregate - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometHashAggregate - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.store_returns - : : : +- ReusedSubquery - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.store - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometNativeScan parquet spark_catalog.default.customer - -Comet accelerated 47 out of 49 eligible operators (95%). Final plan contains 1 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_2/q1.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_2/q1.native_iceberg_compat/extended.txt deleted file mode 100644 index b7c63811ca..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_2/q1.native_iceberg_compat/extended.txt +++ /dev/null @@ -1,53 +0,0 @@ -CometNativeColumnarToRow -+- CometTakeOrderedAndProject - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometHashAggregate - : : : +- CometExchange - : : : +- CometHashAggregate - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns - : : : : +- 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 - : : +- CometHashAggregate - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometHashAggregate - : : +- CometExchange - : : +- CometHashAggregate - : : +- 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 - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer - -Comet accelerated 46 out of 49 eligible operators (93%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_2/q10.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_2/q10.native_datafusion/extended.txt deleted file mode 100644 index 5e8a5364b6..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_2/q10.native_datafusion/extended.txt +++ /dev/null @@ -1,61 +0,0 @@ -TakeOrderedAndProject -+- HashAggregate - +- Exchange - +- HashAggregate - +- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- Project - : : +- Filter - : : +- BroadcastHashJoin - : : :- BroadcastHashJoin [COMET: Unsupported join type ExistenceJoin(exists#1)] - : : : :- CometNativeColumnarToRow - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.customer - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometNativeScan parquet spark_catalog.default.store_sales - : : : : : +- CometSubqueryBroadcast - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometNativeScan parquet spark_catalog.default.web_sales - : : : : +- ReusedSubquery - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : +- ReusedSubquery - : : +- CometBroadcastExchange - : : +- 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 35 out of 54 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_2/q10.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_2/q10.native_iceberg_compat/extended.txt deleted file mode 100644 index 72960fc267..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_2/q10.native_iceberg_compat/extended.txt +++ /dev/null @@ -1,62 +0,0 @@ -TakeOrderedAndProject -+- HashAggregate - +- 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 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_2/q11.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_2/q11.native_datafusion/extended.txt deleted file mode 100644 index 7f06caa2e1..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_2/q11.native_datafusion/extended.txt +++ /dev/null @@ -1,89 +0,0 @@ -CometNativeColumnarToRow -+- CometTakeOrderedAndProject - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometHashAggregate - : : : +- CometExchange - : : : +- CometHashAggregate - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.customer - : : : : +- CometBroadcastExchange - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : : +- CometSubqueryBroadcast - : : : : +- CometBroadcastExchange - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometHashAggregate - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.customer - : : : +- CometBroadcastExchange - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : +- CometSubqueryBroadcast - : : : +- CometBroadcastExchange - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometFilter - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.customer - : : +- CometBroadcastExchange - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : +- ReusedSubquery - : +- CometBroadcastExchange - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.date_dim - +- CometBroadcastExchange - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.customer - : +- CometBroadcastExchange - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.web_sales - : +- ReusedSubquery - +- CometBroadcastExchange - +- CometFilter - +- CometNativeScan parquet spark_catalog.default.date_dim - -Comet accelerated 82 out of 86 eligible operators (95%). Final plan contains 1 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_2/q11.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_2/q11.native_iceberg_compat/extended.txt deleted file mode 100644 index 0e6397bfc2..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_2/q11.native_iceberg_compat/extended.txt +++ /dev/null @@ -1,91 +0,0 @@ -CometNativeColumnarToRow -+- CometTakeOrderedAndProject - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometHashAggregate - : : : +- CometExchange - : : : +- CometHashAggregate - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer - : : : : +- CometBroadcastExchange - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometHashAggregate - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer - : : : +- CometBroadcastExchange - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometFilter - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer - : : +- CometBroadcastExchange - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : +- ReusedSubquery - : +- CometBroadcastExchange - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - +- CometBroadcastExchange - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer - : +- CometBroadcastExchange - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : +- ReusedSubquery - +- CometBroadcastExchange - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - -Comet accelerated 80 out of 86 eligible operators (93%). Final plan contains 3 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_2/q12.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_2/q12.native_datafusion/extended.txt deleted file mode 100644 index a6cc526735..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_2/q12.native_datafusion/extended.txt +++ /dev/null @@ -1,30 +0,0 @@ -TakeOrderedAndProject -+- 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 - :- CometProject - : +- CometBroadcastHashJoin - : :- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : +- CometSubqueryBroadcast - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.item - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometNativeScan parquet spark_catalog.default.date_dim - -Comet accelerated 23 out of 27 eligible operators (85%). Final plan contains 1 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_2/q12.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_2/q12.native_iceberg_compat/extended.txt deleted file mode 100644 index eb85a80eb1..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_2/q12.native_iceberg_compat/extended.txt +++ /dev/null @@ -1,31 +0,0 @@ -TakeOrderedAndProject -+- 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 - :- 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.item - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - -Comet accelerated 22 out of 27 eligible operators (81%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_2/q13.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_2/q13.native_datafusion/extended.txt deleted file mode 100644 index 33c2dd1bbc..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_2/q13.native_datafusion/extended.txt +++ /dev/null @@ -1,41 +0,0 @@ -CometNativeColumnarToRow -+- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : : : +- CometSubqueryBroadcast - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.store - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.customer_address - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.customer_demographics - +- CometBroadcastExchange - +- CometFilter - +- CometNativeScan parquet spark_catalog.default.household_demographics - -Comet accelerated 37 out of 38 eligible operators (97%). Final plan contains 1 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_2/q13.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_2/q13.native_iceberg_compat/extended.txt deleted file mode 100644 index 385e0fdaad..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_2/q13.native_iceberg_compat/extended.txt +++ /dev/null @@ -1,42 +0,0 @@ -CometNativeColumnarToRow -+- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- 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 - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - : : : +- 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.customer_demographics - +- CometBroadcastExchange - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.household_demographics - -Comet accelerated 36 out of 38 eligible operators (94%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_2/q14a.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_2/q14a.native_datafusion/extended.txt deleted file mode 100644 index 1ae3f23b40..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_2/q14a.native_datafusion/extended.txt +++ /dev/null @@ -1,462 +0,0 @@ -CometNativeColumnarToRow -+- CometTakeOrderedAndProject - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometExpand - +- CometUnion - :- CometProject - : +- CometFilter - : : +- Subquery - : : +- CometNativeColumnarToRow - : : +- CometHashAggregate - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometUnion - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometNativeScan parquet spark_catalog.default.store_sales - : : : : +- ReusedSubquery - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : +- ReusedSubquery - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometNativeScan parquet spark_catalog.default.web_sales - : : : +- ReusedSubquery - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : : +- CometSubqueryBroadcast - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : +- CometBroadcastExchange - : : : +- CometBroadcastHashJoin - : : : :- CometHashAggregate - : : : : +- CometExchange - : : : : +- CometHashAggregate - : : : : +- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : : : : +- CometSubqueryBroadcast - : : : : : : +- CometBroadcastExchange - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : : +- CometBroadcastExchange - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometProject - : : : : : : +- CometBroadcastHashJoin - : : : : : : :- CometFilter - : : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : : : : +- ReusedSubquery - : : : : : : +- CometBroadcastExchange - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : : : +- ReusedSubquery - : : : : +- CometBroadcastExchange - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.item - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.item - : : +- CometBroadcastExchange - : : +- CometBroadcastHashJoin - : : :- CometHashAggregate - : : : +- CometExchange - : : : +- CometHashAggregate - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : : : +- CometSubqueryBroadcast - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : : : +- ReusedSubquery - : : : : : +- CometBroadcastExchange - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : : +- ReusedSubquery - : : : +- CometBroadcastExchange - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.item - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.date_dim - :- CometProject - : +- CometFilter - : : +- ReusedSubquery - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : +- ReusedSubquery - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : +- CometBroadcastExchange - : : : +- CometBroadcastHashJoin - : : : :- CometHashAggregate - : : : : +- CometExchange - : : : : +- CometHashAggregate - : : : : +- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : : : : +- CometSubqueryBroadcast - : : : : : : +- CometBroadcastExchange - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : : +- CometBroadcastExchange - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometProject - : : : : : : +- CometBroadcastHashJoin - : : : : : : :- CometFilter - : : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : : : : +- ReusedSubquery - : : : : : : +- CometBroadcastExchange - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : : : +- ReusedSubquery - : : : : +- CometBroadcastExchange - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.item - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.item - : : +- CometBroadcastExchange - : : +- CometBroadcastHashJoin - : : :- CometHashAggregate - : : : +- CometExchange - : : : +- CometHashAggregate - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : : : +- CometSubqueryBroadcast - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : : : +- ReusedSubquery - : : : : : +- CometBroadcastExchange - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : : +- ReusedSubquery - : : : +- CometBroadcastExchange - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.item - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.date_dim - +- CometProject - +- CometFilter - : +- ReusedSubquery - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : +- ReusedSubquery - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.item - : : +- CometBroadcastExchange - : : +- CometBroadcastHashJoin - : : :- CometHashAggregate - : : : +- CometExchange - : : : +- CometHashAggregate - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : : : +- CometSubqueryBroadcast - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : : : +- ReusedSubquery - : : : : : +- CometBroadcastExchange - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : : +- ReusedSubquery - : : : +- CometBroadcastExchange - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.item - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometBroadcastHashJoin - : :- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.item - : +- CometBroadcastExchange - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.item - : +- CometBroadcastExchange - : +- CometBroadcastHashJoin - : :- CometHashAggregate - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : : +- CometSubqueryBroadcast - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : : +- ReusedSubquery - : : : : +- CometBroadcastExchange - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : +- ReusedSubquery - : : +- CometBroadcastExchange - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.item - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.date_dim - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometNativeScan parquet spark_catalog.default.date_dim - -Comet accelerated 431 out of 458 eligible operators (94%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_2/q14a.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_2/q14a.native_iceberg_compat/extended.txt deleted file mode 100644 index 466892ba9d..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_2/q14a.native_iceberg_compat/extended.txt +++ /dev/null @@ -1,469 +0,0 @@ -CometNativeColumnarToRow -+- CometTakeOrderedAndProject - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometExpand - +- CometUnion - :- CometProject - : +- CometFilter - : : +- Subquery - : : +- CometNativeColumnarToRow - : : +- CometHashAggregate - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometUnion - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : +- ReusedSubquery - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : :- 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 - : : +- 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 - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- 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 - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : +- CometBroadcastExchange - : : : +- 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 - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometProject - : : : : : : +- CometBroadcastHashJoin - : : : : : : :- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : : : +- ReusedSubquery - : : : : : : +- CometBroadcastExchange - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : : +- CometBroadcastExchange - : : : : : +- 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 - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : : +- ReusedSubquery - : : : : +- CometBroadcastExchange - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : +- CometBroadcastExchange - : : +- 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 - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : : +- ReusedSubquery - : : : : : +- CometBroadcastExchange - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : +- CometBroadcastExchange - : : : : +- 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 - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : +- ReusedSubquery - : : : +- CometBroadcastExchange - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : +- CometBroadcastExchange - : : +- 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 - :- CometProject - : +- CometFilter - : : +- ReusedSubquery - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : +- ReusedSubquery - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : +- CometBroadcastExchange - : : : +- 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 - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometProject - : : : : : : +- CometBroadcastHashJoin - : : : : : : :- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : : : +- ReusedSubquery - : : : : : : +- CometBroadcastExchange - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : : +- CometBroadcastExchange - : : : : : +- 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 - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : : +- ReusedSubquery - : : : : +- CometBroadcastExchange - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : +- CometBroadcastExchange - : : +- 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 - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : : +- ReusedSubquery - : : : : : +- CometBroadcastExchange - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : +- CometBroadcastExchange - : : : : +- 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 - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : +- ReusedSubquery - : : : +- CometBroadcastExchange - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : +- CometBroadcastExchange - : : +- 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 - +- CometProject - +- CometFilter - : +- ReusedSubquery - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : +- ReusedSubquery - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : +- CometBroadcastExchange - : : +- 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 - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : : +- ReusedSubquery - : : : : : +- CometBroadcastExchange - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : +- CometBroadcastExchange - : : : : +- 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 - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : +- ReusedSubquery - : : : +- CometBroadcastExchange - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometBroadcastHashJoin - : :- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : +- CometBroadcastExchange - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : +- CometBroadcastExchange - : +- 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 - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : +- ReusedSubquery - : : : : +- CometBroadcastExchange - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : +- CometBroadcastExchange - : : : +- 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 - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : +- ReusedSubquery - : : +- CometBroadcastExchange - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : +- CometBroadcastExchange - : +- 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 - -Comet accelerated 424 out of 458 eligible operators (92%). 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_2/q14b.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_2/q14b.native_datafusion/extended.txt deleted file mode 100644 index 85cda8b553..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_2/q14b.native_datafusion/extended.txt +++ /dev/null @@ -1,345 +0,0 @@ -CometNativeColumnarToRow -+- CometTakeOrderedAndProject - +- CometBroadcastHashJoin - :- CometFilter - : : +- Subquery - : : +- CometNativeColumnarToRow - : : +- CometHashAggregate - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometUnion - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometNativeScan parquet spark_catalog.default.store_sales - : : : : +- ReusedSubquery - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : +- ReusedSubquery - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometNativeScan parquet spark_catalog.default.web_sales - : : : +- ReusedSubquery - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : : +- CometSubqueryBroadcast - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : : +- ReusedSubquery - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : +- Subquery - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : +- CometBroadcastExchange - : : : +- CometBroadcastHashJoin - : : : :- CometHashAggregate - : : : : +- CometExchange - : : : : +- CometHashAggregate - : : : : +- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : : : : +- CometSubqueryBroadcast - : : : : : : +- CometBroadcastExchange - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : : +- CometBroadcastExchange - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometProject - : : : : : : +- CometBroadcastHashJoin - : : : : : : :- CometFilter - : : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : : : : +- ReusedSubquery - : : : : : : +- CometBroadcastExchange - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : : : +- ReusedSubquery - : : : : +- CometBroadcastExchange - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.item - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.item - : : +- CometBroadcastExchange - : : +- CometBroadcastHashJoin - : : :- CometHashAggregate - : : : +- CometExchange - : : : +- CometHashAggregate - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : : : +- CometSubqueryBroadcast - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : : : +- ReusedSubquery - : : : : : +- CometBroadcastExchange - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : : +- ReusedSubquery - : : : +- CometBroadcastExchange - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.item - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : : +- ReusedSubquery - : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- Subquery - : +- CometNativeColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.date_dim - +- CometBroadcastExchange - +- CometFilter - : +- ReusedSubquery - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : +- CometSubqueryBroadcast - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : : +- ReusedSubquery - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- Subquery - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.item - : : +- CometBroadcastExchange - : : +- CometBroadcastHashJoin - : : :- CometHashAggregate - : : : +- CometExchange - : : : +- CometHashAggregate - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : : : +- CometSubqueryBroadcast - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : : : +- ReusedSubquery - : : : : : +- CometBroadcastExchange - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : : +- ReusedSubquery - : : : +- CometBroadcastExchange - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.item - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometBroadcastHashJoin - : :- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.item - : +- CometBroadcastExchange - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.item - : +- CometBroadcastExchange - : +- CometBroadcastHashJoin - : :- CometHashAggregate - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : : +- CometSubqueryBroadcast - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : : +- ReusedSubquery - : : : : +- CometBroadcastExchange - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : +- ReusedSubquery - : : +- CometBroadcastExchange - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.item - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.date_dim - +- CometBroadcastExchange - +- CometProject - +- CometFilter - : +- ReusedSubquery - +- CometNativeScan parquet spark_catalog.default.date_dim - +- Subquery - +- CometNativeColumnarToRow - +- CometProject - +- CometFilter - +- CometNativeScan parquet spark_catalog.default.date_dim - -Comet accelerated 310 out of 337 eligible operators (91%). Final plan contains 6 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_2/q14b.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_2/q14b.native_iceberg_compat/extended.txt deleted file mode 100644 index 7a4afd3d2b..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_2/q14b.native_iceberg_compat/extended.txt +++ /dev/null @@ -1,343 +0,0 @@ -CometNativeColumnarToRow -+- CometTakeOrderedAndProject - +- CometBroadcastHashJoin - :- CometFilter - : : +- Subquery - : : +- CometNativeColumnarToRow - : : +- CometHashAggregate - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometUnion - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : +- ReusedSubquery - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : :- 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 - : : +- 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 - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : : +- ReusedSubquery - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- Subquery - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : +- CometBroadcastExchange - : : : +- 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 - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometProject - : : : : : : +- CometBroadcastHashJoin - : : : : : : :- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : : : +- ReusedSubquery - : : : : : : +- CometBroadcastExchange - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : : +- CometBroadcastExchange - : : : : : +- 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 - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : : +- ReusedSubquery - : : : : +- CometBroadcastExchange - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : +- CometBroadcastExchange - : : +- 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 - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : : +- ReusedSubquery - : : : : : +- CometBroadcastExchange - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : +- CometBroadcastExchange - : : : : +- 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 - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : +- ReusedSubquery - : : : +- CometBroadcastExchange - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : : +- ReusedSubquery - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- ReusedSubquery - +- CometBroadcastExchange - +- CometFilter - : +- ReusedSubquery - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : : +- ReusedSubquery - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- Subquery - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : +- CometBroadcastExchange - : : +- 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 - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : : +- ReusedSubquery - : : : : : +- CometBroadcastExchange - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : +- CometBroadcastExchange - : : : : +- 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 - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : +- ReusedSubquery - : : : +- CometBroadcastExchange - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometBroadcastHashJoin - : :- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : +- CometBroadcastExchange - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : +- CometBroadcastExchange - : +- 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 - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : +- ReusedSubquery - : : : : +- CometBroadcastExchange - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : +- CometBroadcastExchange - : : : +- 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 - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : +- ReusedSubquery - : : +- CometBroadcastExchange - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - +- CometBroadcastExchange - +- CometProject - +- CometFilter - : +- ReusedSubquery - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - +- ReusedSubquery - -Comet accelerated 298 out of 331 eligible operators (90%). Final plan contains 10 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_2/q15.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_2/q15.native_datafusion/extended.txt deleted file mode 100644 index da1e91c969..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_2/q15.native_datafusion/extended.txt +++ /dev/null @@ -1,31 +0,0 @@ -CometNativeColumnarToRow -+- CometTakeOrderedAndProject - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : +- CometSubqueryBroadcast - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.customer - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.customer_address - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometNativeScan parquet spark_catalog.default.date_dim - -Comet accelerated 27 out of 28 eligible operators (96%). Final plan contains 1 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_2/q15.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_2/q15.native_iceberg_compat/extended.txt deleted file mode 100644 index b8907128e8..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_2/q15.native_iceberg_compat/extended.txt +++ /dev/null @@ -1,32 +0,0 @@ -CometNativeColumnarToRow -+- CometTakeOrderedAndProject - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_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 - -Comet accelerated 26 out of 28 eligible operators (92%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_2/q16.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_2/q16.native_datafusion/extended.txt deleted file mode 100644 index 97b705f044..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_2/q16.native_datafusion/extended.txt +++ /dev/null @@ -1,42 +0,0 @@ -CometNativeColumnarToRow -+- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometHashAggregate - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometSortMergeJoin - : : : :- CometProject - : : : : +- CometSortMergeJoin - : : : : :- CometSort - : : : : : +- CometExchange - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : +- CometSort - : : : : +- CometExchange - : : : : +- CometProject - : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : +- CometSort - : : : +- CometExchange - : : : +- CometProject - : : : +- CometNativeScan parquet spark_catalog.default.catalog_returns - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.customer_address - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometNativeScan parquet spark_catalog.default.call_center - -Comet accelerated 39 out of 39 eligible operators (100%). Final plan contains 1 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_2/q16.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_2/q16.native_iceberg_compat/extended.txt deleted file mode 100644 index c4efd6fb10..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_2/q16.native_iceberg_compat/extended.txt +++ /dev/null @@ -1,42 +0,0 @@ -CometNativeColumnarToRow -+- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometHashAggregate - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometSortMergeJoin - : : : :- CometProject - : : : : +- CometSortMergeJoin - : : : : :- CometSort - : : : : : +- CometExchange - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : +- CometSort - : : : : +- CometExchange - : : : : +- CometProject - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : +- CometSort - : : : +- CometExchange - : : : +- CometProject - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns - : : +- 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_address - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.call_center - -Comet accelerated 39 out of 39 eligible operators (100%). Final plan contains 1 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_2/q17.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_2/q17.native_datafusion/extended.txt deleted file mode 100644 index a3f43554cf..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_2/q17.native_datafusion/extended.txt +++ /dev/null @@ -1,60 +0,0 @@ -CometNativeColumnarToRow -+- CometTakeOrderedAndProject - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometProject - : : : : : : +- CometBroadcastHashJoin - : : : : : : :- CometFilter - : : : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : : : : : +- CometSubqueryBroadcast - : : : : : : : +- CometBroadcastExchange - : : : : : : : +- CometProject - : : : : : : : +- CometFilter - : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : : : +- CometBroadcastExchange - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.store_returns - : : : : : : +- CometSubqueryBroadcast - : : : : : : +- CometBroadcastExchange - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : : +- CometBroadcastExchange - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : : +- ReusedSubquery - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.store - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometNativeScan parquet spark_catalog.default.item - -Comet accelerated 54 out of 57 eligible operators (94%). Final plan contains 1 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_2/q17.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_2/q17.native_iceberg_compat/extended.txt deleted file mode 100644 index 52ca742891..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_2/q17.native_iceberg_compat/extended.txt +++ /dev/null @@ -1,62 +0,0 @@ -CometNativeColumnarToRow -+- CometTakeOrderedAndProject - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- 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 - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns - : : : : : : +- 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.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.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.store - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - -Comet accelerated 52 out of 57 eligible operators (91%). Final plan contains 3 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_2/q18.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_2/q18.native_datafusion/extended.txt deleted file mode 100644 index 4040a49d5a..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_2/q18.native_datafusion/extended.txt +++ /dev/null @@ -1,50 +0,0 @@ -CometNativeColumnarToRow -+- CometTakeOrderedAndProject - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometExpand - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : : : +- CometSubqueryBroadcast - : : : : : : +- CometBroadcastExchange - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.customer_demographics - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.customer - : : : +- CometBroadcastExchange - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.customer_demographics - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.customer_address - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.date_dim - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometNativeScan parquet spark_catalog.default.item - -Comet accelerated 46 out of 47 eligible operators (97%). Final plan contains 1 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_2/q18.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_2/q18.native_iceberg_compat/extended.txt deleted file mode 100644 index 6ae79a435f..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_2/q18.native_iceberg_compat/extended.txt +++ /dev/null @@ -1,51 +0,0 @@ -CometNativeColumnarToRow -+- CometTakeOrderedAndProject - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometExpand - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_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.customer_demographics - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer - : : : +- CometBroadcastExchange - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics - : : +- 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 - -Comet accelerated 45 out of 47 eligible operators (95%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_2/q19.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_2/q19.native_datafusion/extended.txt deleted file mode 100644 index fa37da7c94..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_2/q19.native_datafusion/extended.txt +++ /dev/null @@ -1,38 +0,0 @@ -CometNativeColumnarToRow -+- CometTakeOrderedAndProject - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.item - : : +- CometBroadcastExchange - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.customer - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.customer_address - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometNativeScan parquet spark_catalog.default.store - -Comet accelerated 35 out of 35 eligible operators (100%). Final plan contains 1 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_2/q19.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_2/q19.native_iceberg_compat/extended.txt deleted file mode 100644 index 9c22a76577..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_2/q19.native_iceberg_compat/extended.txt +++ /dev/null @@ -1,38 +0,0 @@ -CometNativeColumnarToRow -+- CometTakeOrderedAndProject - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : +- 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.store - -Comet accelerated 35 out of 35 eligible operators (100%). Final plan contains 1 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_2/q20.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_2/q20.native_datafusion/extended.txt deleted file mode 100644 index 24498162ca..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_2/q20.native_datafusion/extended.txt +++ /dev/null @@ -1,30 +0,0 @@ -TakeOrderedAndProject -+- 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 - :- CometProject - : +- CometBroadcastHashJoin - : :- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : +- CometSubqueryBroadcast - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.item - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometNativeScan parquet spark_catalog.default.date_dim - -Comet accelerated 23 out of 27 eligible operators (85%). Final plan contains 1 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_2/q20.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_2/q20.native_iceberg_compat/extended.txt deleted file mode 100644 index 643e47b0c6..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_2/q20.native_iceberg_compat/extended.txt +++ /dev/null @@ -1,31 +0,0 @@ -TakeOrderedAndProject -+- 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 - :- CometProject - : +- CometBroadcastHashJoin - : :- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_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.item - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - -Comet accelerated 22 out of 27 eligible operators (81%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_2/q21.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_2/q21.native_datafusion/extended.txt deleted file mode 100644 index 9cbd136255..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_2/q21.native_datafusion/extended.txt +++ /dev/null @@ -1,30 +0,0 @@ -CometNativeColumnarToRow -+- CometTakeOrderedAndProject - +- CometFilter - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.inventory - : : : +- CometSubqueryBroadcast - : : : +- CometBroadcastExchange - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.warehouse - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.item - +- CometBroadcastExchange - +- CometFilter - +- CometNativeScan parquet spark_catalog.default.date_dim - -Comet accelerated 26 out of 27 eligible operators (96%). Final plan contains 1 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_2/q21.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_2/q21.native_iceberg_compat/extended.txt deleted file mode 100644 index 8da34310d7..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_2/q21.native_iceberg_compat/extended.txt +++ /dev/null @@ -1,31 +0,0 @@ -CometNativeColumnarToRow -+- CometTakeOrderedAndProject - +- CometFilter - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.inventory - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.warehouse - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - +- CometBroadcastExchange - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - -Comet accelerated 25 out of 27 eligible operators (92%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_2/q22.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_2/q22.native_datafusion/extended.txt deleted file mode 100644 index 36a0eb0c8a..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_2/q22.native_datafusion/extended.txt +++ /dev/null @@ -1,32 +0,0 @@ -CometNativeColumnarToRow -+- CometTakeOrderedAndProject - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometExpand - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.inventory - : : : +- CometSubqueryBroadcast - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.item - +- CometBroadcastExchange - +- CometFilter - +- CometNativeScan parquet spark_catalog.default.warehouse - -Comet accelerated 28 out of 29 eligible operators (96%). Final plan contains 1 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_2/q22.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_2/q22.native_iceberg_compat/extended.txt deleted file mode 100644 index 831d892584..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_2/q22.native_iceberg_compat/extended.txt +++ /dev/null @@ -1,33 +0,0 @@ -CometNativeColumnarToRow -+- CometTakeOrderedAndProject - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometExpand - +- CometProject - +- CometBroadcastHashJoin - :- 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 - +- CometBroadcastExchange - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.warehouse - -Comet accelerated 27 out of 29 eligible operators (93%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_2/q23a.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_2/q23a.native_datafusion/extended.txt deleted file mode 100644 index 6dd0348345..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_2/q23a.native_datafusion/extended.txt +++ /dev/null @@ -1,142 +0,0 @@ -CometNativeColumnarToRow -+- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometUnion - :- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometSortMergeJoin - : : :- CometSort - : : : +- CometExchange - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : +- CometSubqueryBroadcast - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometHashAggregate - : : : +- CometExchange - : : : +- CometHashAggregate - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : : : +- CometSubqueryBroadcast - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.item - : : +- CometSort - : : +- CometProject - : : +- CometFilter - : : : +- Subquery - : : : +- CometNativeColumnarToRow - : : : +- CometHashAggregate - : : : +- CometExchange - : : : +- CometHashAggregate - : : : +- CometHashAggregate - : : : +- CometExchange - : : : +- CometHashAggregate - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : : : +- CometSubqueryBroadcast - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.customer - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- CometHashAggregate - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : +- CometBroadcastExchange - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.customer - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.date_dim - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometSortMergeJoin - : :- CometSort - : : +- CometExchange - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometNativeScan parquet spark_catalog.default.web_sales - : : : +- ReusedSubquery - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometHashAggregate - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : : +- CometSubqueryBroadcast - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.item - : +- CometSort - : +- CometProject - : +- CometFilter - : : +- ReusedSubquery - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.store_sales - : +- CometBroadcastExchange - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.customer - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometNativeScan parquet spark_catalog.default.date_dim - -Comet accelerated 131 out of 138 eligible operators (94%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_2/q23a.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_2/q23a.native_iceberg_compat/extended.txt deleted file mode 100644 index 94af831deb..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_2/q23a.native_iceberg_compat/extended.txt +++ /dev/null @@ -1,146 +0,0 @@ -CometNativeColumnarToRow -+- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometUnion - :- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometSortMergeJoin - : : :- CometSort - : : : +- CometExchange - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- 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.item - : : +- CometSort - : : +- CometProject - : : +- CometFilter - : : : +- Subquery - : : : +- CometNativeColumnarToRow - : : : +- CometHashAggregate - : : : +- CometExchange - : : : +- CometHashAggregate - : : : +- 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 - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometHashAggregate - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : +- CometBroadcastExchange - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometSortMergeJoin - : :- CometSort - : : +- CometExchange - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : +- ReusedSubquery - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- 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.item - : +- CometSort - : +- CometProject - : +- CometFilter - : : +- ReusedSubquery - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : +- CometBroadcastExchange - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - -Comet accelerated 127 out of 138 eligible operators (92%). 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_2/q23b.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_2/q23b.native_datafusion/extended.txt deleted file mode 100644 index f2228c277c..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_2/q23b.native_datafusion/extended.txt +++ /dev/null @@ -1,194 +0,0 @@ -CometNativeColumnarToRow -+- CometTakeOrderedAndProject - +- CometUnion - :- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometSortMergeJoin - : : : :- CometSort - : : : : +- CometExchange - : : : : +- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : : +- CometSubqueryBroadcast - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometHashAggregate - : : : : +- CometExchange - : : : : +- CometHashAggregate - : : : : +- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : : : : +- CometSubqueryBroadcast - : : : : : : +- CometBroadcastExchange - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : +- CometSort - : : : +- CometProject - : : : +- CometFilter - : : : : +- Subquery - : : : : +- CometNativeColumnarToRow - : : : : +- CometHashAggregate - : : : : +- CometExchange - : : : : +- CometHashAggregate - : : : : +- CometHashAggregate - : : : : +- CometExchange - : : : : +- CometHashAggregate - : : : : +- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : : : : +- CometSubqueryBroadcast - : : : : : : +- CometBroadcastExchange - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : : +- CometBroadcastExchange - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.customer - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- CometHashAggregate - : : : +- CometExchange - : : : +- CometHashAggregate - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : +- CometBroadcastExchange - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.customer - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometSortMergeJoin - : : :- CometSort - : : : +- CometExchange - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.customer - : : +- CometSort - : : +- CometProject - : : +- CometFilter - : : : +- ReusedSubquery - : : +- CometHashAggregate - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : +- CometBroadcastExchange - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.customer - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.date_dim - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometSortMergeJoin - : : :- CometSort - : : : +- CometExchange - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : : +- ReusedSubquery - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometHashAggregate - : : : +- CometExchange - : : : +- CometHashAggregate - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : : : +- CometSubqueryBroadcast - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.item - : : +- CometSort - : : +- CometProject - : : +- CometFilter - : : : +- ReusedSubquery - : : +- CometHashAggregate - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : +- CometBroadcastExchange - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.customer - : +- CometBroadcastExchange - : +- CometProject - : +- CometSortMergeJoin - : :- CometSort - : : +- CometExchange - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.customer - : +- CometSort - : +- CometProject - : +- CometFilter - : : +- ReusedSubquery - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.store_sales - : +- CometBroadcastExchange - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.customer - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometNativeScan parquet spark_catalog.default.date_dim - -Comet accelerated 181 out of 190 eligible operators (95%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_2/q23b.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_2/q23b.native_iceberg_compat/extended.txt deleted file mode 100644 index 2cff47fb04..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_2/q23b.native_iceberg_compat/extended.txt +++ /dev/null @@ -1,198 +0,0 @@ -CometNativeColumnarToRow -+- CometTakeOrderedAndProject - +- CometUnion - :- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometSortMergeJoin - : : : :- CometSort - : : : : +- CometExchange - : : : : +- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- 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.item - : : : +- CometSort - : : : +- CometProject - : : : +- CometFilter - : : : : +- Subquery - : : : : +- CometNativeColumnarToRow - : : : : +- CometHashAggregate - : : : : +- CometExchange - : : : : +- CometHashAggregate - : : : : +- 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 - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometHashAggregate - : : : +- CometExchange - : : : +- CometHashAggregate - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : +- CometBroadcastExchange - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometSortMergeJoin - : : :- CometSort - : : : +- CometExchange - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer - : : +- CometSort - : : +- CometProject - : : +- CometFilter - : : : +- ReusedSubquery - : : +- CometHashAggregate - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : +- CometBroadcastExchange - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometSortMergeJoin - : : :- CometSort - : : : +- CometExchange - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : +- ReusedSubquery - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- 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.item - : : +- CometSort - : : +- CometProject - : : +- CometFilter - : : : +- ReusedSubquery - : : +- CometHashAggregate - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : +- CometBroadcastExchange - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer - : +- CometBroadcastExchange - : +- CometProject - : +- CometSortMergeJoin - : :- CometSort - : : +- CometExchange - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer - : +- CometSort - : +- CometProject - : +- CometFilter - : : +- ReusedSubquery - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : +- CometBroadcastExchange - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - -Comet accelerated 177 out of 190 eligible operators (93%). Final plan contains 6 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_2/q24a.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_2/q24a.native_datafusion/extended.txt deleted file mode 100644 index 8249b86a1e..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_2/q24a.native_datafusion/extended.txt +++ /dev/null @@ -1,92 +0,0 @@ -Filter -: +- Subquery -: +- HashAggregate -: +- Exchange -: +- HashAggregate -: +- HashAggregate -: +- 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 - +- Exchange - +- HashAggregate - +- HashAggregate - +- 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 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_2/q24a.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_2/q24a.native_iceberg_compat/extended.txt deleted file mode 100644 index fa983a36da..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_2/q24a.native_iceberg_compat/extended.txt +++ /dev/null @@ -1,92 +0,0 @@ -Filter -: +- Subquery -: +- HashAggregate -: +- Exchange -: +- HashAggregate -: +- HashAggregate -: +- 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 - +- Exchange - +- HashAggregate - +- HashAggregate - +- 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 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_2/q24b.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_2/q24b.native_datafusion/extended.txt deleted file mode 100644 index 8249b86a1e..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_2/q24b.native_datafusion/extended.txt +++ /dev/null @@ -1,92 +0,0 @@ -Filter -: +- Subquery -: +- HashAggregate -: +- Exchange -: +- HashAggregate -: +- HashAggregate -: +- 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 - +- Exchange - +- HashAggregate - +- HashAggregate - +- 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 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_2/q24b.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_2/q24b.native_iceberg_compat/extended.txt deleted file mode 100644 index fa983a36da..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_2/q24b.native_iceberg_compat/extended.txt +++ /dev/null @@ -1,92 +0,0 @@ -Filter -: +- Subquery -: +- HashAggregate -: +- Exchange -: +- HashAggregate -: +- HashAggregate -: +- 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 - +- Exchange - +- HashAggregate - +- HashAggregate - +- 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 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_2/q25.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_2/q25.native_datafusion/extended.txt deleted file mode 100644 index a3f43554cf..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_2/q25.native_datafusion/extended.txt +++ /dev/null @@ -1,60 +0,0 @@ -CometNativeColumnarToRow -+- CometTakeOrderedAndProject - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometProject - : : : : : : +- CometBroadcastHashJoin - : : : : : : :- CometFilter - : : : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : : : : : +- CometSubqueryBroadcast - : : : : : : : +- CometBroadcastExchange - : : : : : : : +- CometProject - : : : : : : : +- CometFilter - : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : : : +- CometBroadcastExchange - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.store_returns - : : : : : : +- CometSubqueryBroadcast - : : : : : : +- CometBroadcastExchange - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : : +- CometBroadcastExchange - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : : +- ReusedSubquery - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.store - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometNativeScan parquet spark_catalog.default.item - -Comet accelerated 54 out of 57 eligible operators (94%). Final plan contains 1 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_2/q25.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_2/q25.native_iceberg_compat/extended.txt deleted file mode 100644 index 52ca742891..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_2/q25.native_iceberg_compat/extended.txt +++ /dev/null @@ -1,62 +0,0 @@ -CometNativeColumnarToRow -+- CometTakeOrderedAndProject - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- 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 - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns - : : : : : : +- 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.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.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.store - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - -Comet accelerated 52 out of 57 eligible operators (91%). Final plan contains 3 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_2/q26.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_2/q26.native_datafusion/extended.txt deleted file mode 100644 index 2f7fe7b668..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_2/q26.native_datafusion/extended.txt +++ /dev/null @@ -1,38 +0,0 @@ -CometNativeColumnarToRow -+- CometTakeOrderedAndProject - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : +- CometSubqueryBroadcast - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.customer_demographics - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.item - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometNativeScan parquet spark_catalog.default.promotion - -Comet accelerated 34 out of 35 eligible operators (97%). Final plan contains 1 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_2/q26.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_2/q26.native_iceberg_compat/extended.txt deleted file mode 100644 index d868ceeeb3..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_2/q26.native_iceberg_compat/extended.txt +++ /dev/null @@ -1,39 +0,0 @@ -CometNativeColumnarToRow -+- CometTakeOrderedAndProject - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_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.customer_demographics - : : +- 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 - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.promotion - -Comet accelerated 33 out of 35 eligible operators (94%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_2/q27.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_2/q27.native_datafusion/extended.txt deleted file mode 100644 index df27dd568f..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_2/q27.native_datafusion/extended.txt +++ /dev/null @@ -1,39 +0,0 @@ -CometNativeColumnarToRow -+- CometTakeOrderedAndProject - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometExpand - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : : +- CometSubqueryBroadcast - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.customer_demographics - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.store - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometNativeScan parquet spark_catalog.default.item - -Comet accelerated 35 out of 36 eligible operators (97%). Final plan contains 1 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_2/q27.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_2/q27.native_iceberg_compat/extended.txt deleted file mode 100644 index 8b6297f645..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_2/q27.native_iceberg_compat/extended.txt +++ /dev/null @@ -1,40 +0,0 @@ -CometNativeColumnarToRow -+- CometTakeOrderedAndProject - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometExpand - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- 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.customer_demographics - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - -Comet accelerated 34 out of 36 eligible operators (94%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_2/q28.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_2/q28.native_datafusion/extended.txt deleted file mode 100644 index 6dbeba7e9a..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_2/q28.native_datafusion/extended.txt +++ /dev/null @@ -1,72 +0,0 @@ -BroadcastNestedLoopJoin -:- BroadcastNestedLoopJoin -: :- BroadcastNestedLoopJoin -: : :- BroadcastNestedLoopJoin -: : : :- BroadcastNestedLoopJoin [COMET: BroadcastNestedLoopJoin is not supported] -: : : : :- CometNativeColumnarToRow -: : : : : +- CometHashAggregate -: : : : : +- CometExchange -: : : : : +- CometHashAggregate -: : : : : +- CometHashAggregate -: : : : : +- CometExchange -: : : : : +- CometHashAggregate -: : : : : +- CometProject -: : : : : +- CometFilter -: : : : : +- CometNativeScan parquet spark_catalog.default.store_sales -: : : : +- BroadcastExchange -: : : : +- CometNativeColumnarToRow -: : : : +- CometHashAggregate -: : : : +- CometExchange -: : : : +- CometHashAggregate -: : : : +- CometHashAggregate -: : : : +- CometExchange -: : : : +- CometHashAggregate -: : : : +- CometProject -: : : : +- CometFilter -: : : : +- CometNativeScan parquet spark_catalog.default.store_sales -: : : +- BroadcastExchange -: : : +- CometNativeColumnarToRow -: : : +- CometHashAggregate -: : : +- CometExchange -: : : +- CometHashAggregate -: : : +- CometHashAggregate -: : : +- CometExchange -: : : +- CometHashAggregate -: : : +- CometProject -: : : +- CometFilter -: : : +- CometNativeScan parquet spark_catalog.default.store_sales -: : +- BroadcastExchange -: : +- CometNativeColumnarToRow -: : +- CometHashAggregate -: : +- CometExchange -: : +- CometHashAggregate -: : +- CometHashAggregate -: : +- CometExchange -: : +- CometHashAggregate -: : +- CometProject -: : +- CometFilter -: : +- CometNativeScan parquet spark_catalog.default.store_sales -: +- BroadcastExchange -: +- CometNativeColumnarToRow -: +- CometHashAggregate -: +- CometExchange -: +- CometHashAggregate -: +- CometHashAggregate -: +- CometExchange -: +- CometHashAggregate -: +- CometProject -: +- CometFilter -: +- CometNativeScan parquet spark_catalog.default.store_sales -+- BroadcastExchange - +- CometNativeColumnarToRow - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometFilter - +- CometNativeScan parquet spark_catalog.default.store_sales - -Comet accelerated 54 out of 64 eligible operators (84%). 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_2/q28.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_2/q28.native_iceberg_compat/extended.txt deleted file mode 100644 index 35dac9a909..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_2/q28.native_iceberg_compat/extended.txt +++ /dev/null @@ -1,72 +0,0 @@ -BroadcastNestedLoopJoin -:- BroadcastNestedLoopJoin -: :- BroadcastNestedLoopJoin -: : :- BroadcastNestedLoopJoin -: : : :- BroadcastNestedLoopJoin [COMET: BroadcastNestedLoopJoin is not supported] -: : : : :- CometNativeColumnarToRow -: : : : : +- CometHashAggregate -: : : : : +- CometExchange -: : : : : +- CometHashAggregate -: : : : : +- CometHashAggregate -: : : : : +- CometExchange -: : : : : +- CometHashAggregate -: : : : : +- CometProject -: : : : : +- CometFilter -: : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -: : : : +- BroadcastExchange -: : : : +- CometNativeColumnarToRow -: : : : +- CometHashAggregate -: : : : +- CometExchange -: : : : +- CometHashAggregate -: : : : +- CometHashAggregate -: : : : +- CometExchange -: : : : +- CometHashAggregate -: : : : +- CometProject -: : : : +- CometFilter -: : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -: : : +- BroadcastExchange -: : : +- CometNativeColumnarToRow -: : : +- CometHashAggregate -: : : +- CometExchange -: : : +- CometHashAggregate -: : : +- CometHashAggregate -: : : +- CometExchange -: : : +- CometHashAggregate -: : : +- CometProject -: : : +- CometFilter -: : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -: : +- BroadcastExchange -: : +- CometNativeColumnarToRow -: : +- CometHashAggregate -: : +- CometExchange -: : +- CometHashAggregate -: : +- CometHashAggregate -: : +- CometExchange -: : +- CometHashAggregate -: : +- CometProject -: : +- CometFilter -: : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -: +- BroadcastExchange -: +- CometNativeColumnarToRow -: +- CometHashAggregate -: +- CometExchange -: +- CometHashAggregate -: +- CometHashAggregate -: +- CometExchange -: +- CometHashAggregate -: +- CometProject -: +- CometFilter -: +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -+- BroadcastExchange - +- CometNativeColumnarToRow - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - -Comet accelerated 54 out of 64 eligible operators (84%). 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_2/q29.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_2/q29.native_datafusion/extended.txt deleted file mode 100644 index 2650cd8f1a..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_2/q29.native_datafusion/extended.txt +++ /dev/null @@ -1,64 +0,0 @@ -CometNativeColumnarToRow -+- CometTakeOrderedAndProject - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometProject - : : : : : : +- CometBroadcastHashJoin - : : : : : : :- CometFilter - : : : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : : : : : +- CometSubqueryBroadcast - : : : : : : : +- CometBroadcastExchange - : : : : : : : +- CometProject - : : : : : : : +- CometFilter - : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : : : +- CometBroadcastExchange - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.store_returns - : : : : : : +- CometSubqueryBroadcast - : : : : : : +- CometBroadcastExchange - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : : +- CometBroadcastExchange - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : : +- CometSubqueryBroadcast - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.store - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometNativeScan parquet spark_catalog.default.item - -Comet accelerated 58 out of 61 eligible operators (95%). Final plan contains 1 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_2/q29.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_2/q29.native_iceberg_compat/extended.txt deleted file mode 100644 index 9ca9f72ab4..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_2/q29.native_iceberg_compat/extended.txt +++ /dev/null @@ -1,67 +0,0 @@ -CometNativeColumnarToRow -+- CometTakeOrderedAndProject - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- 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 - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns - : : : : : : +- 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.catalog_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.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.store - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - -Comet accelerated 55 out of 61 eligible operators (90%). Final plan contains 4 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_2/q3.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_2/q3.native_datafusion/extended.txt deleted file mode 100644 index b65033bdd7..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_2/q3.native_datafusion/extended.txt +++ /dev/null @@ -1,21 +0,0 @@ -CometNativeColumnarToRow -+- CometTakeOrderedAndProject - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.store_sales - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometNativeScan parquet spark_catalog.default.item - -Comet accelerated 18 out of 18 eligible operators (100%). Final plan contains 1 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_2/q3.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_2/q3.native_iceberg_compat/extended.txt deleted file mode 100644 index e5122dc8d7..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_2/q3.native_iceberg_compat/extended.txt +++ /dev/null @@ -1,21 +0,0 @@ -CometNativeColumnarToRow -+- CometTakeOrderedAndProject - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - -Comet accelerated 18 out of 18 eligible operators (100%). Final plan contains 1 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_2/q30.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_2/q30.native_datafusion/extended.txt deleted file mode 100644 index 3f88234deb..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_2/q30.native_datafusion/extended.txt +++ /dev/null @@ -1,64 +0,0 @@ -CometNativeColumnarToRow -+- CometTakeOrderedAndProject - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometHashAggregate - : : : +- CometExchange - : : : +- CometHashAggregate - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.web_returns - : : : : : +- CometSubqueryBroadcast - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.customer_address - : : +- CometBroadcastExchange - : : +- CometFilter - : : +- CometHashAggregate - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometHashAggregate - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.web_returns - : : : : +- ReusedSubquery - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.customer_address - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.customer - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometNativeScan parquet spark_catalog.default.customer_address - -Comet accelerated 59 out of 61 eligible operators (96%). Final plan contains 1 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_2/q30.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_2/q30.native_iceberg_compat/extended.txt deleted file mode 100644 index 8b2f687037..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_2/q30.native_iceberg_compat/extended.txt +++ /dev/null @@ -1,65 +0,0 @@ -CometNativeColumnarToRow -+- CometTakeOrderedAndProject - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometHashAggregate - : : : +- CometExchange - : : : +- CometHashAggregate - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns - : : : : : +- 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_address - : : +- CometBroadcastExchange - : : +- CometFilter - : : +- CometHashAggregate - : : +- CometExchange - : : +- CometHashAggregate - : : +- 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 - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address - -Comet accelerated 58 out of 61 eligible operators (95%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_2/q31.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_2/q31.native_datafusion/extended.txt deleted file mode 100644 index 188e553f1a..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_2/q31.native_datafusion/extended.txt +++ /dev/null @@ -1,123 +0,0 @@ -CometNativeColumnarToRow -+- CometSort - +- CometExchange - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometBroadcastHashJoin - : : : : :- CometHashAggregate - : : : : : +- CometExchange - : : : : : +- CometHashAggregate - : : : : : +- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometProject - : : : : : : +- CometBroadcastHashJoin - : : : : : : :- CometFilter - : : : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : : : : : +- CometSubqueryBroadcast - : : : : : : : +- CometBroadcastExchange - : : : : : : : +- CometFilter - : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : : : +- CometBroadcastExchange - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : : +- CometBroadcastExchange - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.customer_address - : : : : +- CometBroadcastExchange - : : : : +- CometHashAggregate - : : : : +- CometExchange - : : : : +- CometHashAggregate - : : : : +- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : : : : +- CometSubqueryBroadcast - : : : : : : +- CometBroadcastExchange - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : : +- CometBroadcastExchange - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.customer_address - : : : +- CometBroadcastExchange - : : : +- CometHashAggregate - : : : +- CometExchange - : : : +- CometHashAggregate - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : : : +- CometSubqueryBroadcast - : : : : : +- CometBroadcastExchange - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.customer_address - : : +- CometBroadcastExchange - : : +- CometHashAggregate - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : : +- ReusedSubquery - : : : +- CometBroadcastExchange - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.customer_address - : +- CometBroadcastExchange - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : +- ReusedSubquery - : : +- CometBroadcastExchange - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.customer_address - +- CometBroadcastExchange - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : +- ReusedSubquery - : +- CometBroadcastExchange - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.date_dim - +- CometBroadcastExchange - +- CometFilter - +- CometNativeScan parquet spark_catalog.default.customer_address - -Comet accelerated 114 out of 120 eligible operators (95%). Final plan contains 1 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_2/q31.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_2/q31.native_iceberg_compat/extended.txt deleted file mode 100644 index 3318935f4d..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_2/q31.native_iceberg_compat/extended.txt +++ /dev/null @@ -1,126 +0,0 @@ -CometNativeColumnarToRow -+- CometSort - +- CometExchange - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometBroadcastHashJoin - : : : : :- CometHashAggregate - : : : : : +- CometExchange - : : : : : +- CometHashAggregate - : : : : : +- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometProject - : : : : : : +- CometBroadcastHashJoin - : : : : : : :- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : : : +- SubqueryBroadcast - : : : : : : : +- BroadcastExchange - : : : : : : : +- CometNativeColumnarToRow - : : : : : : : +- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : : : +- CometBroadcastExchange - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : : +- CometBroadcastExchange - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address - : : : : +- CometBroadcastExchange - : : : : +- CometHashAggregate - : : : : +- CometExchange - : : : : +- CometHashAggregate - : : : : +- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometNativeColumnarToRow - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : : +- CometBroadcastExchange - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address - : : : +- CometBroadcastExchange - : : : +- CometHashAggregate - : : : +- CometExchange - : : : +- CometHashAggregate - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address - : : +- CometBroadcastExchange - : : +- CometHashAggregate - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : +- ReusedSubquery - : : : +- CometBroadcastExchange - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address - : +- CometBroadcastExchange - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : +- ReusedSubquery - : : +- CometBroadcastExchange - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address - +- CometBroadcastExchange - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : +- ReusedSubquery - : +- CometBroadcastExchange - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - +- CometBroadcastExchange - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address - -Comet accelerated 111 out of 120 eligible operators (92%). Final plan contains 4 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_2/q32.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_2/q32.native_datafusion/extended.txt deleted file mode 100644 index a536c4b38f..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_2/q32.native_datafusion/extended.txt +++ /dev/null @@ -1,41 +0,0 @@ -CometNativeColumnarToRow -+- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : +- CometSubqueryBroadcast - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.item - : +- CometBroadcastExchange - : +- CometFilter - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : +- ReusedSubquery - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.date_dim - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometNativeScan parquet spark_catalog.default.date_dim - -Comet accelerated 36 out of 38 eligible operators (94%). Final plan contains 1 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_2/q32.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_2/q32.native_iceberg_compat/extended.txt deleted file mode 100644 index 60c49121a6..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_2/q32.native_iceberg_compat/extended.txt +++ /dev/null @@ -1,42 +0,0 @@ -CometNativeColumnarToRow -+- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_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.item - : +- CometBroadcastExchange - : +- CometFilter - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- 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.date_dim - -Comet accelerated 35 out of 38 eligible operators (92%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_2/q33.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_2/q33.native_datafusion/extended.txt deleted file mode 100644 index 7d65fff0b9..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_2/q33.native_datafusion/extended.txt +++ /dev/null @@ -1,95 +0,0 @@ -CometNativeColumnarToRow -+- CometTakeOrderedAndProject - +- CometHashAggregate - +- CometHashAggregate - +- CometUnion - :- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : : +- CometSubqueryBroadcast - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.customer_address - : +- CometBroadcastExchange - : +- CometBroadcastHashJoin - : :- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.item - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.item - :- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : +- ReusedSubquery - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.customer_address - : +- CometBroadcastExchange - : +- CometBroadcastHashJoin - : :- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.item - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.item - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : +- ReusedSubquery - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.customer_address - +- CometBroadcastExchange - +- CometBroadcastHashJoin - :- CometFilter - : +- CometNativeScan parquet spark_catalog.default.item - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometNativeScan parquet spark_catalog.default.item - -Comet accelerated 89 out of 92 eligible operators (96%). Final plan contains 1 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_2/q33.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_2/q33.native_iceberg_compat/extended.txt deleted file mode 100644 index 93698f3e40..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_2/q33.native_iceberg_compat/extended.txt +++ /dev/null @@ -1,96 +0,0 @@ -CometNativeColumnarToRow -+- CometTakeOrderedAndProject - +- CometHashAggregate - +- CometHashAggregate - +- CometUnion - :- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- 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_address - : +- CometBroadcastExchange - : +- CometBroadcastHashJoin - : :- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - :- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- 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_address - : +- CometBroadcastExchange - : +- CometBroadcastHashJoin - : :- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- 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_address - +- CometBroadcastExchange - +- CometBroadcastHashJoin - :- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - -Comet accelerated 88 out of 92 eligible operators (95%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_2/q34.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_2/q34.native_datafusion/extended.txt deleted file mode 100644 index 4c0ef9abcd..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_2/q34.native_datafusion/extended.txt +++ /dev/null @@ -1,40 +0,0 @@ -CometNativeColumnarToRow -+- CometSort - +- CometExchange - +- CometProject - +- CometBroadcastHashJoin - :- CometFilter - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : : +- CometSubqueryBroadcast - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.store - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.household_demographics - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometNativeScan parquet spark_catalog.default.customer - -Comet accelerated 36 out of 37 eligible operators (97%). Final plan contains 1 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_2/q34.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_2/q34.native_iceberg_compat/extended.txt deleted file mode 100644 index 18ed3e6335..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_2/q34.native_iceberg_compat/extended.txt +++ /dev/null @@ -1,41 +0,0 @@ -CometNativeColumnarToRow -+- CometSort - +- CometExchange - +- CometProject - +- CometBroadcastHashJoin - :- CometFilter - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- 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.store - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.household_demographics - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer - -Comet accelerated 35 out of 37 eligible operators (94%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_2/q35.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_2/q35.native_datafusion/extended.txt deleted file mode 100644 index 5e8a5364b6..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_2/q35.native_datafusion/extended.txt +++ /dev/null @@ -1,61 +0,0 @@ -TakeOrderedAndProject -+- HashAggregate - +- Exchange - +- HashAggregate - +- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- Project - : : +- Filter - : : +- BroadcastHashJoin - : : :- BroadcastHashJoin [COMET: Unsupported join type ExistenceJoin(exists#1)] - : : : :- CometNativeColumnarToRow - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.customer - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometNativeScan parquet spark_catalog.default.store_sales - : : : : : +- CometSubqueryBroadcast - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometNativeScan parquet spark_catalog.default.web_sales - : : : : +- ReusedSubquery - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : +- ReusedSubquery - : : +- CometBroadcastExchange - : : +- 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 35 out of 54 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_2/q35.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_2/q35.native_iceberg_compat/extended.txt deleted file mode 100644 index 72960fc267..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_2/q35.native_iceberg_compat/extended.txt +++ /dev/null @@ -1,62 +0,0 @@ -TakeOrderedAndProject -+- HashAggregate - +- 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 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_2/q36.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_2/q36.native_datafusion/extended.txt deleted file mode 100644 index 9480047251..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_2/q36.native_datafusion/extended.txt +++ /dev/null @@ -1,37 +0,0 @@ -TakeOrderedAndProject -+- 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 - +- CometExpand - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : +- CometSubqueryBroadcast - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.item - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometNativeScan parquet spark_catalog.default.store - -Comet accelerated 30 out of 34 eligible operators (88%). Final plan contains 1 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_2/q36.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_2/q36.native_iceberg_compat/extended.txt deleted file mode 100644 index e434c5ff76..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_2/q36.native_iceberg_compat/extended.txt +++ /dev/null @@ -1,38 +0,0 @@ -TakeOrderedAndProject -+- 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 - +- CometExpand - +- CometProject - +- CometBroadcastHashJoin - :- 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.item - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - -Comet accelerated 29 out of 34 eligible operators (85%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_2/q37.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_2/q37.native_datafusion/extended.txt deleted file mode 100644 index a83c218727..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_2/q37.native_datafusion/extended.txt +++ /dev/null @@ -1,33 +0,0 @@ -CometNativeColumnarToRow -+- CometTakeOrderedAndProject - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometBroadcastExchange - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.item - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.inventory - : : +- CometSubqueryBroadcast - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.date_dim - +- CometProject - +- CometFilter - +- CometNativeScan parquet spark_catalog.default.catalog_sales - -Comet accelerated 29 out of 30 eligible operators (96%). Final plan contains 1 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_2/q37.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_2/q37.native_iceberg_compat/extended.txt deleted file mode 100644 index a92eb1903d..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_2/q37.native_iceberg_compat/extended.txt +++ /dev/null @@ -1,34 +0,0 @@ -CometNativeColumnarToRow -+- CometTakeOrderedAndProject - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometBroadcastExchange - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : +- CometBroadcastExchange - : : +- CometProject - : : +- 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 - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - -Comet accelerated 28 out of 30 eligible operators (93%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_2/q38.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_2/q38.native_datafusion/extended.txt deleted file mode 100644 index c965be9cc3..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_2/q38.native_datafusion/extended.txt +++ /dev/null @@ -1,69 +0,0 @@ -CometNativeColumnarToRow -+- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometBroadcastHashJoin - : :- CometHashAggregate - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : : +- CometSubqueryBroadcast - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.customer - : +- CometBroadcastExchange - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : +- ReusedSubquery - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.customer - +- CometBroadcastExchange - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : +- ReusedSubquery - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.date_dim - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometNativeScan parquet spark_catalog.default.customer - -Comet accelerated 63 out of 66 eligible operators (95%). Final plan contains 1 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_2/q38.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_2/q38.native_iceberg_compat/extended.txt deleted file mode 100644 index cb4d06350b..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_2/q38.native_iceberg_compat/extended.txt +++ /dev/null @@ -1,70 +0,0 @@ -CometNativeColumnarToRow -+- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- 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 - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer - : +- CometBroadcastExchange - : +- 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 - +- CometBroadcastExchange - +- 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 62 out of 66 eligible operators (93%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_2/q39a.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_2/q39a.native_datafusion/extended.txt deleted file mode 100644 index 67ff950c02..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_2/q39a.native_datafusion/extended.txt +++ /dev/null @@ -1,63 +0,0 @@ -CometNativeColumnarToRow -+- CometSort - +- CometExchange - +- CometBroadcastHashJoin - :- CometProject - : +- CometFilter - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.inventory - : : : : +- CometSubqueryBroadcast - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.item - : : +- CometBroadcastExchange - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.warehouse - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.date_dim - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.inventory - : : : +- CometSubqueryBroadcast - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.item - : +- CometBroadcastExchange - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.warehouse - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometNativeScan parquet spark_catalog.default.date_dim - -Comet accelerated 58 out of 60 eligible operators (96%). Final plan contains 1 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_2/q39a.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_2/q39a.native_iceberg_compat/extended.txt deleted file mode 100644 index e8f621d32a..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_2/q39a.native_iceberg_compat/extended.txt +++ /dev/null @@ -1,65 +0,0 @@ -CometNativeColumnarToRow -+- CometSort - +- CometExchange - +- CometBroadcastHashJoin - :- CometProject - : +- CometFilter - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- 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 - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : +- CometBroadcastExchange - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.warehouse - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- 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 - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : +- CometBroadcastExchange - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.warehouse - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - -Comet accelerated 56 out of 60 eligible operators (93%). Final plan contains 3 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_2/q39b.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_2/q39b.native_datafusion/extended.txt deleted file mode 100644 index 67ff950c02..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_2/q39b.native_datafusion/extended.txt +++ /dev/null @@ -1,63 +0,0 @@ -CometNativeColumnarToRow -+- CometSort - +- CometExchange - +- CometBroadcastHashJoin - :- CometProject - : +- CometFilter - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.inventory - : : : : +- CometSubqueryBroadcast - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.item - : : +- CometBroadcastExchange - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.warehouse - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.date_dim - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.inventory - : : : +- CometSubqueryBroadcast - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.item - : +- CometBroadcastExchange - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.warehouse - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometNativeScan parquet spark_catalog.default.date_dim - -Comet accelerated 58 out of 60 eligible operators (96%). Final plan contains 1 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_2/q39b.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_2/q39b.native_iceberg_compat/extended.txt deleted file mode 100644 index e8f621d32a..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_2/q39b.native_iceberg_compat/extended.txt +++ /dev/null @@ -1,65 +0,0 @@ -CometNativeColumnarToRow -+- CometSort - +- CometExchange - +- CometBroadcastHashJoin - :- CometProject - : +- CometFilter - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- 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 - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : +- CometBroadcastExchange - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.warehouse - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- 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 - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : +- CometBroadcastExchange - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.warehouse - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - -Comet accelerated 56 out of 60 eligible operators (93%). Final plan contains 3 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_2/q4.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_2/q4.native_datafusion/extended.txt deleted file mode 100644 index ff66588e1a..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_2/q4.native_datafusion/extended.txt +++ /dev/null @@ -1,129 +0,0 @@ -CometNativeColumnarToRow -+- CometTakeOrderedAndProject - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometHashAggregate - : : : : : +- CometExchange - : : : : : +- CometHashAggregate - : : : : : +- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometProject - : : : : : : +- CometBroadcastHashJoin - : : : : : : :- CometProject - : : : : : : : +- CometFilter - : : : : : : : +- CometNativeScan parquet spark_catalog.default.customer - : : : : : : +- CometBroadcastExchange - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : : : : +- CometSubqueryBroadcast - : : : : : : +- CometBroadcastExchange - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : : +- CometBroadcastExchange - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometHashAggregate - : : : : +- CometExchange - : : : : +- CometHashAggregate - : : : : +- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.customer - : : : : : +- CometBroadcastExchange - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : : : +- CometSubqueryBroadcast - : : : : : +- CometBroadcastExchange - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometFilter - : : : +- CometHashAggregate - : : : +- CometExchange - : : : +- CometHashAggregate - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.customer - : : : : +- CometBroadcastExchange - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : +- ReusedSubquery - : : : +- CometBroadcastExchange - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometHashAggregate - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.customer - : : : +- CometBroadcastExchange - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : +- ReusedSubquery - : : +- CometBroadcastExchange - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometFilter - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.customer - : : +- CometBroadcastExchange - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : +- ReusedSubquery - : +- CometBroadcastExchange - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.date_dim - +- CometBroadcastExchange - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.customer - : +- CometBroadcastExchange - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.web_sales - : +- ReusedSubquery - +- CometBroadcastExchange - +- CometFilter - +- CometNativeScan parquet spark_catalog.default.date_dim - -Comet accelerated 120 out of 126 eligible operators (95%). Final plan contains 1 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_2/q4.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_2/q4.native_iceberg_compat/extended.txt deleted file mode 100644 index 570f9bbcc9..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_2/q4.native_iceberg_compat/extended.txt +++ /dev/null @@ -1,131 +0,0 @@ -CometNativeColumnarToRow -+- CometTakeOrderedAndProject - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometHashAggregate - : : : : : +- CometExchange - : : : : : +- CometHashAggregate - : : : : : +- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometProject - : : : : : : +- CometBroadcastHashJoin - : : : : : : :- CometProject - : : : : : : : +- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer - : : : : : : +- CometBroadcastExchange - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometNativeColumnarToRow - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : : +- CometBroadcastExchange - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometHashAggregate - : : : : +- CometExchange - : : : : +- CometHashAggregate - : : : : +- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer - : : : : : +- CometBroadcastExchange - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometFilter - : : : +- CometHashAggregate - : : : +- CometExchange - : : : +- CometHashAggregate - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer - : : : : +- CometBroadcastExchange - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : +- ReusedSubquery - : : : +- CometBroadcastExchange - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometHashAggregate - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer - : : : +- CometBroadcastExchange - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : +- ReusedSubquery - : : +- CometBroadcastExchange - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometFilter - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer - : : +- CometBroadcastExchange - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : +- ReusedSubquery - : +- CometBroadcastExchange - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - +- CometBroadcastExchange - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer - : +- CometBroadcastExchange - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : +- ReusedSubquery - +- CometBroadcastExchange - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - -Comet accelerated 118 out of 126 eligible operators (93%). Final plan contains 3 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_2/q40.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_2/q40.native_datafusion/extended.txt deleted file mode 100644 index 1a148d7ab4..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_2/q40.native_datafusion/extended.txt +++ /dev/null @@ -1,39 +0,0 @@ -CometNativeColumnarToRow -+- CometTakeOrderedAndProject - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometSortMergeJoin - : : : :- CometSort - : : : : +- CometExchange - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : +- CometSubqueryBroadcast - : : : : +- CometBroadcastExchange - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- CometSort - : : : +- CometExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.catalog_returns - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.warehouse - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.item - +- CometBroadcastExchange - +- CometFilter - +- CometNativeScan parquet spark_catalog.default.date_dim - -Comet accelerated 35 out of 36 eligible operators (97%). Final plan contains 1 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_2/q40.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_2/q40.native_iceberg_compat/extended.txt deleted file mode 100644 index f9c807168e..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_2/q40.native_iceberg_compat/extended.txt +++ /dev/null @@ -1,40 +0,0 @@ -CometNativeColumnarToRow -+- CometTakeOrderedAndProject - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometSortMergeJoin - : : : :- CometSort - : : : : +- CometExchange - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometSort - : : : +- CometExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.warehouse - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - +- CometBroadcastExchange - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - -Comet accelerated 34 out of 36 eligible operators (94%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_2/q41.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_2/q41.native_datafusion/extended.txt deleted file mode 100644 index 6f7d077cf3..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_2/q41.native_datafusion/extended.txt +++ /dev/null @@ -1,21 +0,0 @@ -CometNativeColumnarToRow -+- CometTakeOrderedAndProject - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.item - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometFilter - +- CometNativeScan parquet spark_catalog.default.item - -Comet accelerated 18 out of 18 eligible operators (100%). Final plan contains 1 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_2/q41.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_2/q41.native_iceberg_compat/extended.txt deleted file mode 100644 index a3c8dd9568..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_2/q41.native_iceberg_compat/extended.txt +++ /dev/null @@ -1,21 +0,0 @@ -CometNativeColumnarToRow -+- CometTakeOrderedAndProject - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - -Comet accelerated 18 out of 18 eligible operators (100%). Final plan contains 1 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_2/q42.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_2/q42.native_datafusion/extended.txt deleted file mode 100644 index b65033bdd7..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_2/q42.native_datafusion/extended.txt +++ /dev/null @@ -1,21 +0,0 @@ -CometNativeColumnarToRow -+- CometTakeOrderedAndProject - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.store_sales - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometNativeScan parquet spark_catalog.default.item - -Comet accelerated 18 out of 18 eligible operators (100%). Final plan contains 1 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_2/q42.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_2/q42.native_iceberg_compat/extended.txt deleted file mode 100644 index e5122dc8d7..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_2/q42.native_iceberg_compat/extended.txt +++ /dev/null @@ -1,21 +0,0 @@ -CometNativeColumnarToRow -+- CometTakeOrderedAndProject - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - -Comet accelerated 18 out of 18 eligible operators (100%). Final plan contains 1 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_2/q43.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_2/q43.native_datafusion/extended.txt deleted file mode 100644 index cebd2e28d8..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_2/q43.native_datafusion/extended.txt +++ /dev/null @@ -1,21 +0,0 @@ -CometNativeColumnarToRow -+- CometTakeOrderedAndProject - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.store_sales - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometNativeScan parquet spark_catalog.default.store - -Comet accelerated 18 out of 18 eligible operators (100%). Final plan contains 1 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_2/q43.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_2/q43.native_iceberg_compat/extended.txt deleted file mode 100644 index 1272f205e4..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_2/q43.native_iceberg_compat/extended.txt +++ /dev/null @@ -1,21 +0,0 @@ -CometNativeColumnarToRow -+- CometTakeOrderedAndProject - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - -Comet accelerated 18 out of 18 eligible operators (100%). Final plan contains 1 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_2/q44.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_2/q44.native_datafusion/extended.txt deleted file mode 100644 index c1b85fc86e..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_2/q44.native_datafusion/extended.txt +++ /dev/null @@ -1,65 +0,0 @@ -CometNativeColumnarToRow -+- CometTakeOrderedAndProject - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometSortMergeJoin - : : :- CometSort - : : : +- CometColumnarExchange - : : : +- Project - : : : +- Filter - : : : +- Window - : : : +- WindowGroupLimit [COMET: WindowGroupLimit is not supported] - : : : +- CometNativeColumnarToRow - : : : +- CometSort - : : : +- CometColumnarExchange - : : : +- WindowGroupLimit [COMET: WindowGroupLimit is not supported] - : : : +- CometNativeColumnarToRow - : : : +- CometSort - : : : +- CometFilter - : : : : +- Subquery - : : : : +- CometNativeColumnarToRow - : : : : +- CometHashAggregate - : : : : +- CometExchange - : : : : +- CometHashAggregate - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : +- CometHashAggregate - : : : +- CometExchange - : : : +- CometHashAggregate - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : +- CometSort - : : +- CometColumnarExchange - : : +- Project - : : +- Filter - : : +- Window - : : +- WindowGroupLimit [COMET: WindowGroupLimit is not supported] - : : +- CometNativeColumnarToRow - : : +- CometSort - : : +- CometColumnarExchange - : : +- WindowGroupLimit [COMET: WindowGroupLimit is not supported] - : : +- CometNativeColumnarToRow - : : +- CometSort - : : +- CometFilter - : : : +- ReusedSubquery - : : +- CometHashAggregate - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.store_sales - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.item - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometNativeScan parquet spark_catalog.default.item - -Comet accelerated 45 out of 57 eligible operators (78%). 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_2/q44.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_2/q44.native_iceberg_compat/extended.txt deleted file mode 100644 index f7a30e9f90..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_2/q44.native_iceberg_compat/extended.txt +++ /dev/null @@ -1,65 +0,0 @@ -CometNativeColumnarToRow -+- CometTakeOrderedAndProject - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometSortMergeJoin - : : :- CometSort - : : : +- CometColumnarExchange - : : : +- Project - : : : +- Filter - : : : +- Window - : : : +- WindowGroupLimit [COMET: WindowGroupLimit is not supported] - : : : +- CometNativeColumnarToRow - : : : +- CometSort - : : : +- CometColumnarExchange - : : : +- WindowGroupLimit [COMET: WindowGroupLimit is not supported] - : : : +- CometNativeColumnarToRow - : : : +- CometSort - : : : +- CometFilter - : : : : +- Subquery - : : : : +- CometNativeColumnarToRow - : : : : +- CometHashAggregate - : : : : +- CometExchange - : : : : +- CometHashAggregate - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : +- CometHashAggregate - : : : +- CometExchange - : : : +- CometHashAggregate - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : +- CometSort - : : +- CometColumnarExchange - : : +- Project - : : +- Filter - : : +- Window - : : +- WindowGroupLimit [COMET: WindowGroupLimit is not supported] - : : +- CometNativeColumnarToRow - : : +- CometSort - : : +- CometColumnarExchange - : : +- WindowGroupLimit [COMET: WindowGroupLimit is not supported] - : : +- CometNativeColumnarToRow - : : +- CometSort - : : +- CometFilter - : : : +- ReusedSubquery - : : +- CometHashAggregate - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - -Comet accelerated 45 out of 57 eligible operators (78%). 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_2/q45.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_2/q45.native_datafusion/extended.txt deleted file mode 100644 index 6ed69e641f..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_2/q45.native_datafusion/extended.txt +++ /dev/null @@ -1,45 +0,0 @@ -TakeOrderedAndProject -+- HashAggregate - +- Exchange - +- HashAggregate - +- Project - +- Filter - +- BroadcastHashJoin [COMET: Unsupported join type ExistenceJoin(exists#1)] - :- CometNativeColumnarToRow - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : : : +- CometSubqueryBroadcast - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.customer - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.customer_address - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.item - +- BroadcastExchange - +- CometNativeColumnarToRow - +- CometProject - +- CometFilter - +- CometNativeScan parquet spark_catalog.default.item - -Comet accelerated 32 out of 41 eligible operators (78%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_2/q45.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_2/q45.native_iceberg_compat/extended.txt deleted file mode 100644 index f8c94605ab..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_2/q45.native_iceberg_compat/extended.txt +++ /dev/null @@ -1,46 +0,0 @@ -TakeOrderedAndProject -+- HashAggregate - +- 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 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_2/q46.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_2/q46.native_datafusion/extended.txt deleted file mode 100644 index f58213e7cb..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_2/q46.native_datafusion/extended.txt +++ /dev/null @@ -1,48 +0,0 @@ -CometNativeColumnarToRow -+- CometTakeOrderedAndProject - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometHashAggregate - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : : : : +- CometSubqueryBroadcast - : : : : : : +- CometBroadcastExchange - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.store - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.household_demographics - : : +- CometBroadcastExchange - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.customer_address - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.customer - +- CometBroadcastExchange - +- CometFilter - +- CometNativeScan parquet spark_catalog.default.customer_address - -Comet accelerated 44 out of 45 eligible operators (97%). Final plan contains 1 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_2/q46.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_2/q46.native_iceberg_compat/extended.txt deleted file mode 100644 index 07093ae00b..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_2/q46.native_iceberg_compat/extended.txt +++ /dev/null @@ -1,49 +0,0 @@ -CometNativeColumnarToRow -+- CometTakeOrderedAndProject - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometHashAggregate - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- 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.store - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.household_demographics - : : +- CometBroadcastExchange - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer - +- CometBroadcastExchange - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address - -Comet accelerated 43 out of 45 eligible operators (95%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_2/q47.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_2/q47.native_datafusion/extended.txt deleted file mode 100644 index 54fa1daf2e..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_2/q47.native_datafusion/extended.txt +++ /dev/null @@ -1,102 +0,0 @@ -TakeOrderedAndProject -+- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- Project - : : +- Filter - : : +- Window - : : +- 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 - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : +- CometBroadcastExchange - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : : +- CometSubqueryBroadcast - : : : : +- CometBroadcastExchange - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.store - : +- BroadcastExchange - : +- Project - : +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] - : +- CometNativeColumnarToRow - : +- CometSort - : +- CometExchange - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : +- CometBroadcastExchange - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : +- CometSubqueryBroadcast - : : : +- CometBroadcastExchange - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.store - +- BroadcastExchange - +- Project - +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] - +- CometNativeColumnarToRow - +- CometSort - +- CometExchange - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.item - : : +- CometBroadcastExchange - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : +- CometSubqueryBroadcast - : : +- CometBroadcastExchange - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.date_dim - +- CometBroadcastExchange - +- CometFilter - +- CometNativeScan parquet spark_catalog.default.store - -Comet accelerated 78 out of 97 eligible operators (80%). Final plan contains 3 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_2/q47.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_2/q47.native_iceberg_compat/extended.txt deleted file mode 100644 index 9691120d2f..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_2/q47.native_iceberg_compat/extended.txt +++ /dev/null @@ -1,105 +0,0 @@ -TakeOrderedAndProject -+- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- Project - : : +- Filter - : : +- Window - : : +- 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 - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : +- CometBroadcastExchange - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - : +- BroadcastExchange - : +- Project - : +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] - : +- CometNativeColumnarToRow - : +- CometSort - : +- CometExchange - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : +- CometBroadcastExchange - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - +- BroadcastExchange - +- Project - +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] - +- CometNativeColumnarToRow - +- CometSort - +- CometExchange - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : +- CometBroadcastExchange - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - +- CometBroadcastExchange - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - -Comet accelerated 75 out of 97 eligible operators (77%). Final plan contains 6 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_2/q48.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_2/q48.native_datafusion/extended.txt deleted file mode 100644 index 4b1541c674..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_2/q48.native_datafusion/extended.txt +++ /dev/null @@ -1,36 +0,0 @@ -CometNativeColumnarToRow -+- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : : +- CometSubqueryBroadcast - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.store - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.customer_demographics - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.customer_address - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometNativeScan parquet spark_catalog.default.date_dim - -Comet accelerated 32 out of 33 eligible operators (96%). Final plan contains 1 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_2/q48.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_2/q48.native_iceberg_compat/extended.txt deleted file mode 100644 index c8325b49ca..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_2/q48.native_iceberg_compat/extended.txt +++ /dev/null @@ -1,37 +0,0 @@ -CometNativeColumnarToRow -+- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- 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 - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics - : +- 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 - -Comet accelerated 31 out of 33 eligible operators (93%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_2/q49.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_2/q49.native_datafusion/extended.txt deleted file mode 100644 index 138d841ed0..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_2/q49.native_datafusion/extended.txt +++ /dev/null @@ -1,91 +0,0 @@ -TakeOrderedAndProject -+- HashAggregate - +- HashAggregate - +- Union - :- Project - : +- Filter - : +- Window - : +- Sort - : +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set 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 - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : +- CometSubqueryBroadcast - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.web_returns - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.date_dim - :- Project - : +- Filter - : +- Window - : +- Sort - : +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set 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 - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : +- ReusedSubquery - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.catalog_returns - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.date_dim - +- Project - +- Filter - +- Window - +- Sort - +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set 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 - :- CometProject - : +- CometBroadcastHashJoin - : :- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : +- ReusedSubquery - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.store_returns - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometNativeScan parquet spark_catalog.default.date_dim - -Comet accelerated 64 out of 86 eligible operators (74%). 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_2/q49.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_2/q49.native_iceberg_compat/extended.txt deleted file mode 100644 index 2044df0e8a..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_2/q49.native_iceberg_compat/extended.txt +++ /dev/null @@ -1,92 +0,0 @@ -TakeOrderedAndProject -+- HashAggregate - +- HashAggregate - +- Union - :- Project - : +- Filter - : +- Window - : +- Sort - : +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set 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 - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometBroadcastExchange - : : : +- CometProject - : : : +- 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 - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - :- Project - : +- Filter - : +- Window - : +- Sort - : +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set 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 - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : +- ReusedSubquery - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - +- Project - +- Filter - +- Window - +- Sort - +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set 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 - :- CometProject - : +- CometBroadcastHashJoin - : :- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : +- ReusedSubquery - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - -Comet accelerated 63 out of 86 eligible operators (73%). 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_2/q50.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_2/q50.native_datafusion/extended.txt deleted file mode 100644 index e20ab4a6d6..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_2/q50.native_datafusion/extended.txt +++ /dev/null @@ -1,36 +0,0 @@ -CometNativeColumnarToRow -+- CometTakeOrderedAndProject - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : +- CometBroadcastExchange - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.store_returns - : : : +- CometSubqueryBroadcast - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.store - : +- CometBroadcastExchange - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.date_dim - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometNativeScan parquet spark_catalog.default.date_dim - -Comet accelerated 32 out of 33 eligible operators (96%). Final plan contains 1 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_2/q50.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_2/q50.native_iceberg_compat/extended.txt deleted file mode 100644 index aa0c4c1c38..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_2/q50.native_iceberg_compat/extended.txt +++ /dev/null @@ -1,37 +0,0 @@ -CometNativeColumnarToRow -+- CometTakeOrderedAndProject - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : +- CometBroadcastExchange - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns - : : : +- 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.store - : +- CometBroadcastExchange - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - -Comet accelerated 31 out of 33 eligible operators (93%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_2/q51.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_2/q51.native_datafusion/extended.txt deleted file mode 100644 index f67067735c..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_2/q51.native_datafusion/extended.txt +++ /dev/null @@ -1,52 +0,0 @@ -TakeOrderedAndProject -+- 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 - +- CometExchange - +- CometProject - +- CometSortMergeJoin - :- CometSort - : +- CometColumnarExchange - : +- Project - : +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] - : +- CometNativeColumnarToRow - : +- CometSort - : +- CometExchange - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : +- CometSubqueryBroadcast - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.date_dim - +- CometSort - +- CometColumnarExchange - +- Project - +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] - +- CometNativeColumnarToRow - +- CometSort - +- CometExchange - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometFilter - : +- CometNativeScan parquet spark_catalog.default.store_sales - : +- ReusedSubquery - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometNativeScan parquet spark_catalog.default.date_dim - -Comet accelerated 38 out of 47 eligible operators (80%). Final plan contains 3 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_2/q51.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_2/q51.native_iceberg_compat/extended.txt deleted file mode 100644 index ea746c5fee..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_2/q51.native_iceberg_compat/extended.txt +++ /dev/null @@ -1,53 +0,0 @@ -TakeOrderedAndProject -+- 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 - +- CometExchange - +- CometProject - +- CometSortMergeJoin - :- CometSort - : +- CometColumnarExchange - : +- Project - : +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] - : +- CometNativeColumnarToRow - : +- CometSort - : +- 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 - +- 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 37 out of 47 eligible operators (78%). Final plan contains 4 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_2/q52.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_2/q52.native_datafusion/extended.txt deleted file mode 100644 index b65033bdd7..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_2/q52.native_datafusion/extended.txt +++ /dev/null @@ -1,21 +0,0 @@ -CometNativeColumnarToRow -+- CometTakeOrderedAndProject - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.store_sales - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometNativeScan parquet spark_catalog.default.item - -Comet accelerated 18 out of 18 eligible operators (100%). Final plan contains 1 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_2/q52.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_2/q52.native_iceberg_compat/extended.txt deleted file mode 100644 index e5122dc8d7..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_2/q52.native_iceberg_compat/extended.txt +++ /dev/null @@ -1,21 +0,0 @@ -CometNativeColumnarToRow -+- CometTakeOrderedAndProject - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - -Comet accelerated 18 out of 18 eligible operators (100%). Final plan contains 1 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_2/q53.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_2/q53.native_datafusion/extended.txt deleted file mode 100644 index 9a8a9bbba4..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_2/q53.native_datafusion/extended.txt +++ /dev/null @@ -1,36 +0,0 @@ -TakeOrderedAndProject -+- 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 - +- CometExchange - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.item - : : +- CometBroadcastExchange - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : +- CometSubqueryBroadcast - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.date_dim - +- CometBroadcastExchange - +- CometFilter - +- CometNativeScan parquet spark_catalog.default.store - -Comet accelerated 28 out of 33 eligible operators (84%). Final plan contains 1 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_2/q53.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_2/q53.native_iceberg_compat/extended.txt deleted file mode 100644 index 093505a6cc..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_2/q53.native_iceberg_compat/extended.txt +++ /dev/null @@ -1,37 +0,0 @@ -TakeOrderedAndProject -+- 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 - +- CometExchange - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : +- CometBroadcastExchange - : : +- 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 - -Comet accelerated 27 out of 33 eligible operators (81%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_2/q55.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_2/q55.native_datafusion/extended.txt deleted file mode 100644 index b65033bdd7..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_2/q55.native_datafusion/extended.txt +++ /dev/null @@ -1,21 +0,0 @@ -CometNativeColumnarToRow -+- CometTakeOrderedAndProject - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.store_sales - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometNativeScan parquet spark_catalog.default.item - -Comet accelerated 18 out of 18 eligible operators (100%). Final plan contains 1 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_2/q55.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_2/q55.native_iceberg_compat/extended.txt deleted file mode 100644 index e5122dc8d7..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_2/q55.native_iceberg_compat/extended.txt +++ /dev/null @@ -1,21 +0,0 @@ -CometNativeColumnarToRow -+- CometTakeOrderedAndProject - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - -Comet accelerated 18 out of 18 eligible operators (100%). Final plan contains 1 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_2/q56.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_2/q56.native_datafusion/extended.txt deleted file mode 100644 index bf043f9d60..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_2/q56.native_datafusion/extended.txt +++ /dev/null @@ -1,98 +0,0 @@ -CometNativeColumnarToRow -+- CometTakeOrderedAndProject - +- CometHashAggregate - +- CometHashAggregate - +- CometUnion - :- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : : +- CometSubqueryBroadcast - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.customer_address - : +- CometBroadcastExchange - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.item - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.item - :- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : +- ReusedSubquery - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.customer_address - : +- CometBroadcastExchange - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.item - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.item - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : +- ReusedSubquery - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.customer_address - +- CometBroadcastExchange - +- CometProject - +- CometBroadcastHashJoin - :- CometFilter - : +- CometNativeScan parquet spark_catalog.default.item - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometNativeScan parquet spark_catalog.default.item - -Comet accelerated 92 out of 95 eligible operators (96%). Final plan contains 1 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_2/q56.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_2/q56.native_iceberg_compat/extended.txt deleted file mode 100644 index a4d34c2e29..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_2/q56.native_iceberg_compat/extended.txt +++ /dev/null @@ -1,99 +0,0 @@ -CometNativeColumnarToRow -+- CometTakeOrderedAndProject - +- CometHashAggregate - +- CometHashAggregate - +- CometUnion - :- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- 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_address - : +- CometBroadcastExchange - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - :- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- 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_address - : +- CometBroadcastExchange - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- 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_address - +- CometBroadcastExchange - +- CometProject - +- CometBroadcastHashJoin - :- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - -Comet accelerated 91 out of 95 eligible operators (95%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_2/q57.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_2/q57.native_datafusion/extended.txt deleted file mode 100644 index aa4ccf3246..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_2/q57.native_datafusion/extended.txt +++ /dev/null @@ -1,102 +0,0 @@ -TakeOrderedAndProject -+- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- Project - : : +- Filter - : : +- Window - : : +- 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 - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : +- CometBroadcastExchange - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : +- CometSubqueryBroadcast - : : : : +- CometBroadcastExchange - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.call_center - : +- BroadcastExchange - : +- Project - : +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] - : +- CometNativeColumnarToRow - : +- CometSort - : +- CometExchange - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : +- CometBroadcastExchange - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : +- CometSubqueryBroadcast - : : : +- CometBroadcastExchange - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.call_center - +- BroadcastExchange - +- Project - +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] - +- CometNativeColumnarToRow - +- CometSort - +- CometExchange - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.item - : : +- CometBroadcastExchange - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : +- CometSubqueryBroadcast - : : +- CometBroadcastExchange - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.date_dim - +- CometBroadcastExchange - +- CometFilter - +- CometNativeScan parquet spark_catalog.default.call_center - -Comet accelerated 78 out of 97 eligible operators (80%). Final plan contains 3 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_2/q57.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_2/q57.native_iceberg_compat/extended.txt deleted file mode 100644 index 5e8acff889..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_2/q57.native_iceberg_compat/extended.txt +++ /dev/null @@ -1,105 +0,0 @@ -TakeOrderedAndProject -+- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- Project - : : +- Filter - : : +- Window - : : +- 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 - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : +- CometBroadcastExchange - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.call_center - : +- BroadcastExchange - : +- Project - : +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] - : +- CometNativeColumnarToRow - : +- CometSort - : +- CometExchange - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : +- CometBroadcastExchange - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.call_center - +- BroadcastExchange - +- Project - +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] - +- CometNativeColumnarToRow - +- CometSort - +- CometExchange - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : +- CometBroadcastExchange - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - +- CometBroadcastExchange - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.call_center - -Comet accelerated 75 out of 97 eligible operators (77%). Final plan contains 6 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_2/q58.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_2/q58.native_datafusion/extended.txt deleted file mode 100644 index 6597775c11..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_2/q58.native_datafusion/extended.txt +++ /dev/null @@ -1,115 +0,0 @@ -CometNativeColumnarToRow -+- CometTakeOrderedAndProject - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometFilter - : : +- CometHashAggregate - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : : +- CometSubqueryBroadcast - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : : +- ReusedSubquery - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : +- Subquery - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.item - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : : +- ReusedSubquery - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- Subquery - : : +- CometNativeColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometFilter - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : +- ReusedSubquery - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.item - : +- CometBroadcastExchange - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : : +- ReusedSubquery - : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- Subquery - : +- CometNativeColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.date_dim - +- CometBroadcastExchange - +- CometFilter - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : +- ReusedSubquery - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.item - +- CometBroadcastExchange - +- CometProject - +- CometBroadcastHashJoin - :- CometFilter - : +- CometNativeScan parquet spark_catalog.default.date_dim - +- CometBroadcastExchange - +- CometProject - +- CometFilter - : +- ReusedSubquery - +- CometNativeScan parquet spark_catalog.default.date_dim - +- Subquery - +- CometNativeColumnarToRow - +- CometProject - +- CometFilter - +- CometNativeScan parquet spark_catalog.default.date_dim - -Comet accelerated 97 out of 108 eligible operators (89%). Final plan contains 5 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_2/q58.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_2/q58.native_iceberg_compat/extended.txt deleted file mode 100644 index 7190b29a9b..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_2/q58.native_iceberg_compat/extended.txt +++ /dev/null @@ -1,116 +0,0 @@ -CometNativeColumnarToRow -+- CometTakeOrderedAndProject - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometFilter - : : +- CometHashAggregate - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : : +- ReusedSubquery - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- Subquery - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : : +- ReusedSubquery - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- Subquery - : : +- CometNativeColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometFilter - : +- 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.item - : +- CometBroadcastExchange - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : : +- ReusedSubquery - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- Subquery - : +- CometNativeColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - +- CometBroadcastExchange - +- CometFilter - +- 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.item - +- CometBroadcastExchange - +- CometProject - +- CometBroadcastHashJoin - :- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - +- CometBroadcastExchange - +- CometProject - +- CometFilter - : +- ReusedSubquery - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - +- Subquery - +- CometNativeColumnarToRow - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - -Comet accelerated 96 out of 108 eligible operators (88%). Final plan contains 6 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_2/q59.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_2/q59.native_datafusion/extended.txt deleted file mode 100644 index e23616d099..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_2/q59.native_datafusion/extended.txt +++ /dev/null @@ -1,53 +0,0 @@ -CometNativeColumnarToRow -+- CometTakeOrderedAndProject - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometHashAggregate - : : : +- CometExchange - : : : +- CometHashAggregate - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.store - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.date_dim - +- CometBroadcastExchange - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometHashAggregate - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.store - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometNativeScan parquet spark_catalog.default.date_dim - -Comet accelerated 50 out of 50 eligible operators (100%). Final plan contains 1 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_2/q59.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_2/q59.native_iceberg_compat/extended.txt deleted file mode 100644 index 3594fb8035..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_2/q59.native_iceberg_compat/extended.txt +++ /dev/null @@ -1,53 +0,0 @@ -CometNativeColumnarToRow -+- CometTakeOrderedAndProject - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometHashAggregate - : : : +- CometExchange - : : : +- CometHashAggregate - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- 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 - +- CometBroadcastExchange - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometHashAggregate - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- 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 50 out of 50 eligible operators (100%). Final plan contains 1 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_2/q6.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_2/q6.native_datafusion/extended.txt deleted file mode 100644 index dd68f2476e..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_2/q6.native_datafusion/extended.txt +++ /dev/null @@ -1,65 +0,0 @@ -CometNativeColumnarToRow -+- CometTakeOrderedAndProject - +- CometFilter - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.customer_address - : : : +- CometBroadcastExchange - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.customer - : : +- CometBroadcastExchange - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : +- CometSubqueryBroadcast - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : : +- ReusedSubquery - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- Subquery - : : +- CometNativeColumnarToRow - : : +- CometHashAggregate - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : : +- ReusedSubquery - : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- Subquery - : +- CometNativeColumnarToRow - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.date_dim - +- CometBroadcastExchange - +- CometProject - +- CometBroadcastHashJoin - :- CometFilter - : +- CometNativeScan parquet spark_catalog.default.item - +- CometBroadcastExchange - +- CometFilter - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometFilter - +- CometNativeScan parquet spark_catalog.default.item - -Comet accelerated 55 out of 60 eligible operators (91%). Final plan contains 3 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_2/q6.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_2/q6.native_iceberg_compat/extended.txt deleted file mode 100644 index ee55822981..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_2/q6.native_iceberg_compat/extended.txt +++ /dev/null @@ -1,59 +0,0 @@ -CometNativeColumnarToRow -+- CometTakeOrderedAndProject - +- CometFilter - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address - : : : +- CometBroadcastExchange - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer - : : +- CometBroadcastExchange - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometProject - : : +- CometFilter - : : : +- ReusedSubquery - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- Subquery - : : +- CometNativeColumnarToRow - : : +- CometHashAggregate - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : : +- ReusedSubquery - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- ReusedSubquery - +- CometBroadcastExchange - +- CometProject - +- CometBroadcastHashJoin - :- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - +- CometBroadcastExchange - +- CometFilter - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - -Comet accelerated 48 out of 54 eligible operators (88%). 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_2/q60.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_2/q60.native_datafusion/extended.txt deleted file mode 100644 index bf043f9d60..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_2/q60.native_datafusion/extended.txt +++ /dev/null @@ -1,98 +0,0 @@ -CometNativeColumnarToRow -+- CometTakeOrderedAndProject - +- CometHashAggregate - +- CometHashAggregate - +- CometUnion - :- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : : +- CometSubqueryBroadcast - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.customer_address - : +- CometBroadcastExchange - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.item - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.item - :- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : +- ReusedSubquery - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.customer_address - : +- CometBroadcastExchange - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.item - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.item - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : +- ReusedSubquery - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.customer_address - +- CometBroadcastExchange - +- CometProject - +- CometBroadcastHashJoin - :- CometFilter - : +- CometNativeScan parquet spark_catalog.default.item - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometNativeScan parquet spark_catalog.default.item - -Comet accelerated 92 out of 95 eligible operators (96%). Final plan contains 1 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_2/q60.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_2/q60.native_iceberg_compat/extended.txt deleted file mode 100644 index a4d34c2e29..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_2/q60.native_iceberg_compat/extended.txt +++ /dev/null @@ -1,99 +0,0 @@ -CometNativeColumnarToRow -+- CometTakeOrderedAndProject - +- CometHashAggregate - +- CometHashAggregate - +- CometUnion - :- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- 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_address - : +- CometBroadcastExchange - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - :- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- 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_address - : +- CometBroadcastExchange - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- 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_address - +- CometBroadcastExchange - +- CometProject - +- CometBroadcastHashJoin - :- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - -Comet accelerated 91 out of 95 eligible operators (95%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_2/q61.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_2/q61.native_datafusion/extended.txt deleted file mode 100644 index 39f985df28..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_2/q61.native_datafusion/extended.txt +++ /dev/null @@ -1,87 +0,0 @@ -Project -+- BroadcastNestedLoopJoin [COMET: BroadcastNestedLoopJoin is not supported] - :- CometNativeColumnarToRow - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometProject - : : : : : : +- CometBroadcastHashJoin - : : : : : : :- CometFilter - : : : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : : : : : +- CometSubqueryBroadcast - : : : : : : : +- CometBroadcastExchange - : : : : : : : +- CometProject - : : : : : : : +- CometFilter - : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : : : +- CometBroadcastExchange - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.store - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.promotion - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.customer - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.customer_address - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.item - +- BroadcastExchange - +- CometNativeColumnarToRow - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : : : +- ReusedSubquery - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.store - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.customer - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.customer_address - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometNativeScan parquet spark_catalog.default.item - -Comet accelerated 78 out of 83 eligible operators (93%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_2/q61.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_2/q61.native_iceberg_compat/extended.txt deleted file mode 100644 index 783de88195..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_2/q61.native_iceberg_compat/extended.txt +++ /dev/null @@ -1,88 +0,0 @@ -Project -+- BroadcastNestedLoopJoin [COMET: BroadcastNestedLoopJoin is not supported] - :- CometNativeColumnarToRow - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- 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.store - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.promotion - : : : : +- CometBroadcastExchange - : : : : +- 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.item - +- BroadcastExchange - +- CometNativeColumnarToRow - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- 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 - : : +- 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.item - -Comet accelerated 77 out of 83 eligible operators (92%). Final plan contains 3 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_2/q62.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_2/q62.native_datafusion/extended.txt deleted file mode 100644 index dc88244743..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_2/q62.native_datafusion/extended.txt +++ /dev/null @@ -1,31 +0,0 @@ -CometNativeColumnarToRow -+- CometTakeOrderedAndProject - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : +- CometBroadcastExchange - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.warehouse - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.ship_mode - : +- CometBroadcastExchange - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.web_site - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometNativeScan parquet spark_catalog.default.date_dim - -Comet accelerated 28 out of 28 eligible operators (100%). Final plan contains 1 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_2/q62.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_2/q62.native_iceberg_compat/extended.txt deleted file mode 100644 index dc09a1507f..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_2/q62.native_iceberg_compat/extended.txt +++ /dev/null @@ -1,31 +0,0 @@ -CometNativeColumnarToRow -+- CometTakeOrderedAndProject - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : +- CometBroadcastExchange - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.warehouse - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.ship_mode - : +- CometBroadcastExchange - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_site - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - -Comet accelerated 28 out of 28 eligible operators (100%). Final plan contains 1 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_2/q63.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_2/q63.native_datafusion/extended.txt deleted file mode 100644 index 9a8a9bbba4..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_2/q63.native_datafusion/extended.txt +++ /dev/null @@ -1,36 +0,0 @@ -TakeOrderedAndProject -+- 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 - +- CometExchange - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.item - : : +- CometBroadcastExchange - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : +- CometSubqueryBroadcast - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.date_dim - +- CometBroadcastExchange - +- CometFilter - +- CometNativeScan parquet spark_catalog.default.store - -Comet accelerated 28 out of 33 eligible operators (84%). Final plan contains 1 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_2/q63.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_2/q63.native_iceberg_compat/extended.txt deleted file mode 100644 index 093505a6cc..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_2/q63.native_iceberg_compat/extended.txt +++ /dev/null @@ -1,37 +0,0 @@ -TakeOrderedAndProject -+- 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 - +- CometExchange - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : +- CometBroadcastExchange - : : +- 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 - -Comet accelerated 27 out of 33 eligible operators (81%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_2/q64.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_2/q64.native_datafusion/extended.txt deleted file mode 100644 index 14ffe4eab9..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_2/q64.native_datafusion/extended.txt +++ /dev/null @@ -1,245 +0,0 @@ -CometNativeColumnarToRow -+- CometSort - +- CometExchange - +- CometProject - +- CometSortMergeJoin - :- CometSort - : +- CometExchange - : +- CometHashAggregate - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometProject - : : : : : : +- CometBroadcastHashJoin - : : : : : : :- CometProject - : : : : : : : +- CometBroadcastHashJoin - : : : : : : : :- CometProject - : : : : : : : : +- CometBroadcastHashJoin - : : : : : : : : :- CometProject - : : : : : : : : : +- CometBroadcastHashJoin - : : : : : : : : : :- CometProject - : : : : : : : : : : +- CometBroadcastHashJoin - : : : : : : : : : : :- CometProject - : : : : : : : : : : : +- CometBroadcastHashJoin - : : : : : : : : : : : :- CometProject - : : : : : : : : : : : : +- CometBroadcastHashJoin - : : : : : : : : : : : : :- CometProject - : : : : : : : : : : : : : +- CometBroadcastHashJoin - : : : : : : : : : : : : : :- CometProject - : : : : : : : : : : : : : : +- CometBroadcastHashJoin - : : : : : : : : : : : : : : :- CometProject - : : : : : : : : : : : : : : : +- CometBroadcastHashJoin - : : : : : : : : : : : : : : : :- CometProject - : : : : : : : : : : : : : : : : +- CometSortMergeJoin - : : : : : : : : : : : : : : : : :- CometSort - : : : : : : : : : : : : : : : : : +- CometExchange - : : : : : : : : : : : : : : : : : +- CometProject - : : : : : : : : : : : : : : : : : +- CometBroadcastHashJoin - : : : : : : : : : : : : : : : : : :- CometBroadcastExchange - : : : : : : : : : : : : : : : : : : +- CometFilter - : : : : : : : : : : : : : : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : : : : : : : : : : : : : : : : +- CometSubqueryBroadcast - : : : : : : : : : : : : : : : : : : +- CometBroadcastExchange - : : : : : : : : : : : : : : : : : : +- CometFilter - : : : : : : : : : : : : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : : : : : : : : : : : : : : +- CometProject - : : : : : : : : : : : : : : : : : +- CometFilter - : : : : : : : : : : : : : : : : : +- CometNativeScan parquet spark_catalog.default.store_returns - : : : : : : : : : : : : : : : : +- CometSort - : : : : : : : : : : : : : : : : +- CometProject - : : : : : : : : : : : : : : : : +- CometFilter - : : : : : : : : : : : : : : : : +- CometHashAggregate - : : : : : : : : : : : : : : : : +- CometExchange - : : : : : : : : : : : : : : : : +- CometHashAggregate - : : : : : : : : : : : : : : : : +- CometProject - : : : : : : : : : : : : : : : : +- CometSortMergeJoin - : : : : : : : : : : : : : : : : :- CometSort - : : : : : : : : : : : : : : : : : +- CometExchange - : : : : : : : : : : : : : : : : : +- CometProject - : : : : : : : : : : : : : : : : : +- CometFilter - : : : : : : : : : : : : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : : : : : : : : : : : : : +- CometSort - : : : : : : : : : : : : : : : : +- CometExchange - : : : : : : : : : : : : : : : : +- CometProject - : : : : : : : : : : : : : : : : +- CometFilter - : : : : : : : : : : : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_returns - : : : : : : : : : : : : : : : +- CometBroadcastExchange - : : : : : : : : : : : : : : : +- CometFilter - : : : : : : : : : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : : : : : : : : : : : +- CometBroadcastExchange - : : : : : : : : : : : : : : +- CometProject - : : : : : : : : : : : : : : +- CometFilter - : : : : : : : : : : : : : : +- CometNativeScan parquet spark_catalog.default.store - : : : : : : : : : : : : : +- CometBroadcastExchange - : : : : : : : : : : : : : +- CometFilter - : : : : : : : : : : : : : +- CometNativeScan parquet spark_catalog.default.customer - : : : : : : : : : : : : +- CometBroadcastExchange - : : : : : : : : : : : : +- CometFilter - : : : : : : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : : : : : : : : +- CometBroadcastExchange - : : : : : : : : : : : +- CometFilter - : : : : : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : : : : : : : +- CometBroadcastExchange - : : : : : : : : : : +- CometProject - : : : : : : : : : : +- CometFilter - : : : : : : : : : : +- CometNativeScan parquet spark_catalog.default.customer_demographics - : : : : : : : : : +- CometBroadcastExchange - : : : : : : : : : +- CometProject - : : : : : : : : : +- CometFilter - : : : : : : : : : +- CometNativeScan parquet spark_catalog.default.customer_demographics - : : : : : : : : +- CometBroadcastExchange - : : : : : : : : +- CometFilter - : : : : : : : : +- CometNativeScan parquet spark_catalog.default.promotion - : : : : : : : +- CometBroadcastExchange - : : : : : : : +- CometFilter - : : : : : : : +- CometNativeScan parquet spark_catalog.default.household_demographics - : : : : : : +- CometBroadcastExchange - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.household_demographics - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.customer_address - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.customer_address - : : : +- CometBroadcastExchange - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.income_band - : : +- CometBroadcastExchange - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.income_band - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.item - +- CometSort - +- CometExchange - +- CometHashAggregate - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometProject - : : : : : : +- CometBroadcastHashJoin - : : : : : : :- CometProject - : : : : : : : +- CometBroadcastHashJoin - : : : : : : : :- CometProject - : : : : : : : : +- CometBroadcastHashJoin - : : : : : : : : :- CometProject - : : : : : : : : : +- CometBroadcastHashJoin - : : : : : : : : : :- CometProject - : : : : : : : : : : +- CometBroadcastHashJoin - : : : : : : : : : : :- CometProject - : : : : : : : : : : : +- CometBroadcastHashJoin - : : : : : : : : : : : :- CometProject - : : : : : : : : : : : : +- CometBroadcastHashJoin - : : : : : : : : : : : : :- CometProject - : : : : : : : : : : : : : +- CometBroadcastHashJoin - : : : : : : : : : : : : : :- CometProject - : : : : : : : : : : : : : : +- CometBroadcastHashJoin - : : : : : : : : : : : : : : :- CometProject - : : : : : : : : : : : : : : : +- CometSortMergeJoin - : : : : : : : : : : : : : : : :- CometSort - : : : : : : : : : : : : : : : : +- CometExchange - : : : : : : : : : : : : : : : : +- CometProject - : : : : : : : : : : : : : : : : +- CometBroadcastHashJoin - : : : : : : : : : : : : : : : : :- CometBroadcastExchange - : : : : : : : : : : : : : : : : : +- CometFilter - : : : : : : : : : : : : : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : : : : : : : : : : : : : : : +- CometSubqueryBroadcast - : : : : : : : : : : : : : : : : : +- CometBroadcastExchange - : : : : : : : : : : : : : : : : : +- CometFilter - : : : : : : : : : : : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : : : : : : : : : : : : : +- CometProject - : : : : : : : : : : : : : : : : +- CometFilter - : : : : : : : : : : : : : : : : +- CometNativeScan parquet spark_catalog.default.store_returns - : : : : : : : : : : : : : : : +- CometSort - : : : : : : : : : : : : : : : +- CometProject - : : : : : : : : : : : : : : : +- CometFilter - : : : : : : : : : : : : : : : +- CometHashAggregate - : : : : : : : : : : : : : : : +- CometExchange - : : : : : : : : : : : : : : : +- CometHashAggregate - : : : : : : : : : : : : : : : +- CometProject - : : : : : : : : : : : : : : : +- CometSortMergeJoin - : : : : : : : : : : : : : : : :- CometSort - : : : : : : : : : : : : : : : : +- CometExchange - : : : : : : : : : : : : : : : : +- CometProject - : : : : : : : : : : : : : : : : +- CometFilter - : : : : : : : : : : : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : : : : : : : : : : : : +- CometSort - : : : : : : : : : : : : : : : +- CometExchange - : : : : : : : : : : : : : : : +- CometProject - : : : : : : : : : : : : : : : +- CometFilter - : : : : : : : : : : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_returns - : : : : : : : : : : : : : : +- CometBroadcastExchange - : : : : : : : : : : : : : : +- CometFilter - : : : : : : : : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : : : : : : : : : : +- CometBroadcastExchange - : : : : : : : : : : : : : +- CometProject - : : : : : : : : : : : : : +- CometFilter - : : : : : : : : : : : : : +- CometNativeScan parquet spark_catalog.default.store - : : : : : : : : : : : : +- CometBroadcastExchange - : : : : : : : : : : : : +- CometFilter - : : : : : : : : : : : : +- CometNativeScan parquet spark_catalog.default.customer - : : : : : : : : : : : +- CometBroadcastExchange - : : : : : : : : : : : +- CometFilter - : : : : : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : : : : : : : +- CometBroadcastExchange - : : : : : : : : : : +- CometFilter - : : : : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : : : : : : +- CometBroadcastExchange - : : : : : : : : : +- CometProject - : : : : : : : : : +- CometFilter - : : : : : : : : : +- CometNativeScan parquet spark_catalog.default.customer_demographics - : : : : : : : : +- CometBroadcastExchange - : : : : : : : : +- CometProject - : : : : : : : : +- CometFilter - : : : : : : : : +- CometNativeScan parquet spark_catalog.default.customer_demographics - : : : : : : : +- CometBroadcastExchange - : : : : : : : +- CometFilter - : : : : : : : +- CometNativeScan parquet spark_catalog.default.promotion - : : : : : : +- CometBroadcastExchange - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.household_demographics - : : : : : +- CometBroadcastExchange - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.household_demographics - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.customer_address - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.customer_address - : : +- CometBroadcastExchange - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.income_band - : +- CometBroadcastExchange - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.income_band - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometNativeScan parquet spark_catalog.default.item - -Comet accelerated 240 out of 242 eligible operators (99%). Final plan contains 1 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_2/q64.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_2/q64.native_iceberg_compat/extended.txt deleted file mode 100644 index a868670a16..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_2/q64.native_iceberg_compat/extended.txt +++ /dev/null @@ -1,247 +0,0 @@ -CometNativeColumnarToRow -+- CometSort - +- CometExchange - +- CometProject - +- CometSortMergeJoin - :- CometSort - : +- CometExchange - : +- CometHashAggregate - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometProject - : : : : : : +- CometBroadcastHashJoin - : : : : : : :- CometProject - : : : : : : : +- CometBroadcastHashJoin - : : : : : : : :- CometProject - : : : : : : : : +- CometBroadcastHashJoin - : : : : : : : : :- CometProject - : : : : : : : : : +- CometBroadcastHashJoin - : : : : : : : : : :- CometProject - : : : : : : : : : : +- CometBroadcastHashJoin - : : : : : : : : : : :- CometProject - : : : : : : : : : : : +- CometBroadcastHashJoin - : : : : : : : : : : : :- CometProject - : : : : : : : : : : : : +- CometBroadcastHashJoin - : : : : : : : : : : : : :- CometProject - : : : : : : : : : : : : : +- CometBroadcastHashJoin - : : : : : : : : : : : : : :- CometProject - : : : : : : : : : : : : : : +- CometBroadcastHashJoin - : : : : : : : : : : : : : : :- CometProject - : : : : : : : : : : : : : : : +- CometBroadcastHashJoin - : : : : : : : : : : : : : : : :- CometProject - : : : : : : : : : : : : : : : : +- CometSortMergeJoin - : : : : : : : : : : : : : : : : :- CometSort - : : : : : : : : : : : : : : : : : +- CometExchange - : : : : : : : : : : : : : : : : : +- CometProject - : : : : : : : : : : : : : : : : : +- CometBroadcastHashJoin - : : : : : : : : : : : : : : : : : :- CometBroadcastExchange - : : : : : : : : : : : : : : : : : : +- CometFilter - : : : : : : : : : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : : : : : : : : : : : : : : +- SubqueryBroadcast - : : : : : : : : : : : : : : : : : : +- BroadcastExchange - : : : : : : : : : : : : : : : : : : +- CometNativeColumnarToRow - : : : : : : : : : : : : : : : : : : +- CometFilter - : : : : : : : : : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : : : : : : : : : : : : : : +- CometProject - : : : : : : : : : : : : : : : : : +- CometFilter - : : : : : : : : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns - : : : : : : : : : : : : : : : : +- CometSort - : : : : : : : : : : : : : : : : +- CometProject - : : : : : : : : : : : : : : : : +- CometFilter - : : : : : : : : : : : : : : : : +- CometHashAggregate - : : : : : : : : : : : : : : : : +- CometExchange - : : : : : : : : : : : : : : : : +- CometHashAggregate - : : : : : : : : : : : : : : : : +- CometProject - : : : : : : : : : : : : : : : : +- CometSortMergeJoin - : : : : : : : : : : : : : : : : :- CometSort - : : : : : : : : : : : : : : : : : +- CometExchange - : : : : : : : : : : : : : : : : : +- CometProject - : : : : : : : : : : : : : : : : : +- CometFilter - : : : : : : : : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : : : : : : : : : : : : +- CometSort - : : : : : : : : : : : : : : : : +- CometExchange - : : : : : : : : : : : : : : : : +- CometProject - : : : : : : : : : : : : : : : : +- CometFilter - : : : : : : : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns - : : : : : : : : : : : : : : : +- CometBroadcastExchange - : : : : : : : : : : : : : : : +- CometFilter - : : : : : : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : : : : : : : : : : : +- CometBroadcastExchange - : : : : : : : : : : : : : : +- CometProject - : : : : : : : : : : : : : : +- CometFilter - : : : : : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - : : : : : : : : : : : : : +- CometBroadcastExchange - : : : : : : : : : : : : : +- CometFilter - : : : : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer - : : : : : : : : : : : : +- CometBroadcastExchange - : : : : : : : : : : : : +- CometFilter - : : : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : : : : : : : : +- CometBroadcastExchange - : : : : : : : : : : : +- CometFilter - : : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : : : : : : : +- CometBroadcastExchange - : : : : : : : : : : +- CometProject - : : : : : : : : : : +- CometFilter - : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics - : : : : : : : : : +- CometBroadcastExchange - : : : : : : : : : +- CometProject - : : : : : : : : : +- CometFilter - : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics - : : : : : : : : +- CometBroadcastExchange - : : : : : : : : +- CometFilter - : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.promotion - : : : : : : : +- CometBroadcastExchange - : : : : : : : +- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.household_demographics - : : : : : : +- CometBroadcastExchange - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.household_demographics - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address - : : : +- CometBroadcastExchange - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.income_band - : : +- CometBroadcastExchange - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.income_band - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - +- CometSort - +- CometExchange - +- CometHashAggregate - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometProject - : : : : : : +- CometBroadcastHashJoin - : : : : : : :- CometProject - : : : : : : : +- CometBroadcastHashJoin - : : : : : : : :- CometProject - : : : : : : : : +- CometBroadcastHashJoin - : : : : : : : : :- CometProject - : : : : : : : : : +- CometBroadcastHashJoin - : : : : : : : : : :- CometProject - : : : : : : : : : : +- CometBroadcastHashJoin - : : : : : : : : : : :- CometProject - : : : : : : : : : : : +- CometBroadcastHashJoin - : : : : : : : : : : : :- CometProject - : : : : : : : : : : : : +- CometBroadcastHashJoin - : : : : : : : : : : : : :- CometProject - : : : : : : : : : : : : : +- CometBroadcastHashJoin - : : : : : : : : : : : : : :- CometProject - : : : : : : : : : : : : : : +- CometBroadcastHashJoin - : : : : : : : : : : : : : : :- CometProject - : : : : : : : : : : : : : : : +- CometSortMergeJoin - : : : : : : : : : : : : : : : :- CometSort - : : : : : : : : : : : : : : : : +- CometExchange - : : : : : : : : : : : : : : : : +- CometProject - : : : : : : : : : : : : : : : : +- CometBroadcastHashJoin - : : : : : : : : : : : : : : : : :- CometBroadcastExchange - : : : : : : : : : : : : : : : : : +- CometFilter - : : : : : : : : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : : : : : : : : : : : : : +- SubqueryBroadcast - : : : : : : : : : : : : : : : : : +- BroadcastExchange - : : : : : : : : : : : : : : : : : +- CometNativeColumnarToRow - : : : : : : : : : : : : : : : : : +- CometFilter - : : : : : : : : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : : : : : : : : : : : : : +- CometProject - : : : : : : : : : : : : : : : : +- CometFilter - : : : : : : : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns - : : : : : : : : : : : : : : : +- CometSort - : : : : : : : : : : : : : : : +- CometProject - : : : : : : : : : : : : : : : +- CometFilter - : : : : : : : : : : : : : : : +- CometHashAggregate - : : : : : : : : : : : : : : : +- CometExchange - : : : : : : : : : : : : : : : +- CometHashAggregate - : : : : : : : : : : : : : : : +- CometProject - : : : : : : : : : : : : : : : +- CometSortMergeJoin - : : : : : : : : : : : : : : : :- CometSort - : : : : : : : : : : : : : : : : +- CometExchange - : : : : : : : : : : : : : : : : +- CometProject - : : : : : : : : : : : : : : : : +- CometFilter - : : : : : : : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : : : : : : : : : : : +- CometSort - : : : : : : : : : : : : : : : +- CometExchange - : : : : : : : : : : : : : : : +- CometProject - : : : : : : : : : : : : : : : +- CometFilter - : : : : : : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns - : : : : : : : : : : : : : : +- CometBroadcastExchange - : : : : : : : : : : : : : : +- CometFilter - : : : : : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : : : : : : : : : : +- CometBroadcastExchange - : : : : : : : : : : : : : +- CometProject - : : : : : : : : : : : : : +- CometFilter - : : : : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - : : : : : : : : : : : : +- CometBroadcastExchange - : : : : : : : : : : : : +- CometFilter - : : : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer - : : : : : : : : : : : +- CometBroadcastExchange - : : : : : : : : : : : +- CometFilter - : : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : : : : : : : +- CometBroadcastExchange - : : : : : : : : : : +- CometFilter - : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : : : : : : +- CometBroadcastExchange - : : : : : : : : : +- CometProject - : : : : : : : : : +- CometFilter - : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics - : : : : : : : : +- CometBroadcastExchange - : : : : : : : : +- CometProject - : : : : : : : : +- CometFilter - : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics - : : : : : : : +- CometBroadcastExchange - : : : : : : : +- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.promotion - : : : : : : +- CometBroadcastExchange - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.household_demographics - : : : : : +- CometBroadcastExchange - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.household_demographics - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address - : : +- CometBroadcastExchange - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.income_band - : +- CometBroadcastExchange - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.income_band - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - -Comet accelerated 238 out of 242 eligible operators (98%). Final plan contains 3 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_2/q65.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_2/q65.native_datafusion/extended.txt deleted file mode 100644 index b3937f6c97..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_2/q65.native_datafusion/extended.txt +++ /dev/null @@ -1,51 +0,0 @@ -CometNativeColumnarToRow -+- CometTakeOrderedAndProject - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.store - : : +- CometBroadcastExchange - : : +- CometFilter - : : +- CometHashAggregate - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : +- CometSubqueryBroadcast - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.item - +- CometBroadcastExchange - +- CometFilter - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometFilter - : +- CometNativeScan parquet spark_catalog.default.store_sales - : +- ReusedSubquery - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometNativeScan parquet spark_catalog.default.date_dim - -Comet accelerated 46 out of 48 eligible operators (95%). Final plan contains 1 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_2/q65.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_2/q65.native_iceberg_compat/extended.txt deleted file mode 100644 index b0c27d6da7..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_2/q65.native_iceberg_compat/extended.txt +++ /dev/null @@ -1,52 +0,0 @@ -CometNativeColumnarToRow -+- CometTakeOrderedAndProject - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - : : +- CometBroadcastExchange - : : +- CometFilter - : : +- CometHashAggregate - : : +- CometExchange - : : +- CometHashAggregate - : : +- 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.item - +- CometBroadcastExchange - +- CometFilter - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- 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 45 out of 48 eligible operators (93%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_2/q66.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_2/q66.native_datafusion/extended.txt deleted file mode 100644 index a259d59a1b..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_2/q66.native_datafusion/extended.txt +++ /dev/null @@ -1,68 +0,0 @@ -CometNativeColumnarToRow -+- CometTakeOrderedAndProject - +- CometHashAggregate - +- CometHashAggregate - +- CometUnion - :- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : : : +- CometSubqueryBroadcast - : : : : : +- CometBroadcastExchange - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.warehouse - : : : +- CometBroadcastExchange - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.time_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.ship_mode - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : +- ReusedSubquery - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.warehouse - : : +- CometBroadcastExchange - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.time_dim - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometNativeScan parquet spark_catalog.default.ship_mode - -Comet accelerated 63 out of 65 eligible operators (96%). Final plan contains 1 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_2/q66.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_2/q66.native_iceberg_compat/extended.txt deleted file mode 100644 index bbbbb2874f..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_2/q66.native_iceberg_compat/extended.txt +++ /dev/null @@ -1,69 +0,0 @@ -CometNativeColumnarToRow -+- CometTakeOrderedAndProject - +- CometHashAggregate - +- CometHashAggregate - +- CometUnion - :- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.warehouse - : : : +- CometBroadcastExchange - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.time_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.ship_mode - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- 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.warehouse - : : +- CometBroadcastExchange - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.time_dim - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.ship_mode - -Comet accelerated 62 out of 65 eligible operators (95%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_2/q67.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_2/q67.native_datafusion/extended.txt deleted file mode 100644 index 3c1bc08e97..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_2/q67.native_datafusion/extended.txt +++ /dev/null @@ -1,41 +0,0 @@ -TakeOrderedAndProject -+- Filter - +- Window - +- WindowGroupLimit [COMET: WindowGroupLimit is not supported] - +- CometNativeColumnarToRow - +- CometSort - +- CometColumnarExchange - +- WindowGroupLimit [COMET: WindowGroupLimit is not supported] - +- CometNativeColumnarToRow - +- CometSort - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometExpand - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : +- CometSubqueryBroadcast - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.store - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometNativeScan parquet spark_catalog.default.item - -Comet accelerated 31 out of 37 eligible operators (83%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_2/q67.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_2/q67.native_iceberg_compat/extended.txt deleted file mode 100644 index d7fde7133a..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_2/q67.native_iceberg_compat/extended.txt +++ /dev/null @@ -1,42 +0,0 @@ -TakeOrderedAndProject -+- Filter - +- Window - +- WindowGroupLimit [COMET: WindowGroupLimit is not supported] - +- CometNativeColumnarToRow - +- CometSort - +- CometColumnarExchange - +- WindowGroupLimit [COMET: WindowGroupLimit is not supported] - +- CometNativeColumnarToRow - +- CometSort - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometExpand - +- CometProject - +- CometBroadcastHashJoin - :- 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.store - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - -Comet accelerated 30 out of 37 eligible operators (81%). Final plan contains 3 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_2/q68.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_2/q68.native_datafusion/extended.txt deleted file mode 100644 index f58213e7cb..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_2/q68.native_datafusion/extended.txt +++ /dev/null @@ -1,48 +0,0 @@ -CometNativeColumnarToRow -+- CometTakeOrderedAndProject - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometHashAggregate - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : : : : +- CometSubqueryBroadcast - : : : : : : +- CometBroadcastExchange - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.store - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.household_demographics - : : +- CometBroadcastExchange - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.customer_address - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.customer - +- CometBroadcastExchange - +- CometFilter - +- CometNativeScan parquet spark_catalog.default.customer_address - -Comet accelerated 44 out of 45 eligible operators (97%). Final plan contains 1 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_2/q68.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_2/q68.native_iceberg_compat/extended.txt deleted file mode 100644 index 07093ae00b..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_2/q68.native_iceberg_compat/extended.txt +++ /dev/null @@ -1,49 +0,0 @@ -CometNativeColumnarToRow -+- CometTakeOrderedAndProject - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometHashAggregate - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- 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.store - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.household_demographics - : : +- CometBroadcastExchange - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer - +- CometBroadcastExchange - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address - -Comet accelerated 43 out of 45 eligible operators (95%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_2/q69.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_2/q69.native_datafusion/extended.txt deleted file mode 100644 index 41b9fb6b4e..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_2/q69.native_datafusion/extended.txt +++ /dev/null @@ -1,56 +0,0 @@ -CometNativeColumnarToRow -+- CometTakeOrderedAndProject - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometBroadcastHashJoin - : : : :- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.customer - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometNativeScan parquet spark_catalog.default.store_sales - : : : : : +- CometSubqueryBroadcast - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometNativeScan parquet spark_catalog.default.web_sales - : : : : +- ReusedSubquery - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : +- ReusedSubquery - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.customer_address - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometNativeScan parquet spark_catalog.default.customer_demographics - -Comet accelerated 50 out of 53 eligible operators (94%). Final plan contains 1 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_2/q69.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_2/q69.native_iceberg_compat/extended.txt deleted file mode 100644 index e553dcb0a5..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_2/q69.native_iceberg_compat/extended.txt +++ /dev/null @@ -1,57 +0,0 @@ -CometNativeColumnarToRow -+- CometTakeOrderedAndProject - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometBroadcastHashJoin - : : : :- 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 - : : : +- CometBroadcastExchange - : : : +- 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 - : : +- CometBroadcastExchange - : : +- 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 - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics - -Comet accelerated 49 out of 53 eligible operators (92%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_2/q7.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_2/q7.native_datafusion/extended.txt deleted file mode 100644 index 8750081c55..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_2/q7.native_datafusion/extended.txt +++ /dev/null @@ -1,38 +0,0 @@ -CometNativeColumnarToRow -+- CometTakeOrderedAndProject - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : : +- CometSubqueryBroadcast - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.customer_demographics - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.item - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometNativeScan parquet spark_catalog.default.promotion - -Comet accelerated 34 out of 35 eligible operators (97%). Final plan contains 1 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_2/q7.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_2/q7.native_iceberg_compat/extended.txt deleted file mode 100644 index 783b9a64bb..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_2/q7.native_iceberg_compat/extended.txt +++ /dev/null @@ -1,39 +0,0 @@ -CometNativeColumnarToRow -+- CometTakeOrderedAndProject - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- 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.customer_demographics - : : +- 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 - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.promotion - -Comet accelerated 33 out of 35 eligible operators (94%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_2/q70.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_2/q70.native_datafusion/extended.txt deleted file mode 100644 index e5ddbc5b9a..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_2/q70.native_datafusion/extended.txt +++ /dev/null @@ -1,59 +0,0 @@ -TakeOrderedAndProject -+- 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 - +- Expand - +- Project - +- BroadcastHashJoin - :- CometNativeColumnarToRow - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : +- CometSubqueryBroadcast - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- 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 [COMET: WindowGroupLimit is not supported] - +- CometNativeColumnarToRow - +- CometSort - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : +- ReusedSubquery - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.store - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometNativeScan parquet spark_catalog.default.date_dim - -Comet accelerated 34 out of 53 eligible operators (64%). 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_2/q70.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_2/q70.native_iceberg_compat/extended.txt deleted file mode 100644 index 1ce50f8385..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_2/q70.native_iceberg_compat/extended.txt +++ /dev/null @@ -1,60 +0,0 @@ -TakeOrderedAndProject -+- 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 - +- 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 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_2/q71.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_2/q71.native_datafusion/extended.txt deleted file mode 100644 index 2c056d01d7..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_2/q71.native_datafusion/extended.txt +++ /dev/null @@ -1,52 +0,0 @@ -CometNativeColumnarToRow -+- CometSort - +- CometExchange - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.item - : +- CometUnion - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : +- CometSubqueryBroadcast - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : +- ReusedSubquery - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : +- ReusedSubquery - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.date_dim - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometNativeScan parquet spark_catalog.default.time_dim - -Comet accelerated 46 out of 49 eligible operators (93%). Final plan contains 1 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_2/q71.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_2/q71.native_iceberg_compat/extended.txt deleted file mode 100644 index 4de5ab6c3b..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_2/q71.native_iceberg_compat/extended.txt +++ /dev/null @@ -1,53 +0,0 @@ -CometNativeColumnarToRow -+- CometSort - +- CometExchange - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : +- CometUnion - : :- 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 - : :- 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 - : +- 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 - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.time_dim - -Comet accelerated 45 out of 49 eligible operators (91%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_2/q72.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_2/q72.native_datafusion/extended.txt deleted file mode 100644 index d443b31d1e..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_2/q72.native_datafusion/extended.txt +++ /dev/null @@ -1,71 +0,0 @@ -CometNativeColumnarToRow -+- CometTakeOrderedAndProject - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometSortMergeJoin - :- CometSort - : +- CometExchange - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometProject - : : : : : : +- CometBroadcastHashJoin - : : : : : : :- CometProject - : : : : : : : +- CometBroadcastHashJoin - : : : : : : : :- CometProject - : : : : : : : : +- CometBroadcastHashJoin - : : : : : : : : :- CometProject - : : : : : : : : : +- CometBroadcastHashJoin - : : : : : : : : : :- CometFilter - : : : : : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : : : : : : : +- CometSubqueryBroadcast - : : : : : : : : : : +- CometBroadcastExchange - : : : : : : : : : : +- CometProject - : : : : : : : : : : +- CometFilter - : : : : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : : : : : : +- CometBroadcastExchange - : : : : : : : : : +- CometFilter - : : : : : : : : : +- CometNativeScan parquet spark_catalog.default.inventory - : : : : : : : : +- CometBroadcastExchange - : : : : : : : : +- CometFilter - : : : : : : : : +- CometNativeScan parquet spark_catalog.default.warehouse - : : : : : : : +- CometBroadcastExchange - : : : : : : : +- CometFilter - : : : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : : : +- CometBroadcastExchange - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.customer_demographics - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.household_demographics - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.promotion - +- CometSort - +- CometExchange - +- CometProject - +- CometFilter - +- CometNativeScan parquet spark_catalog.default.catalog_returns - -Comet accelerated 67 out of 68 eligible operators (98%). Final plan contains 1 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_2/q72.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_2/q72.native_iceberg_compat/extended.txt deleted file mode 100644 index 22fde3b02c..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_2/q72.native_iceberg_compat/extended.txt +++ /dev/null @@ -1,72 +0,0 @@ -CometNativeColumnarToRow -+- CometTakeOrderedAndProject - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometSortMergeJoin - :- CometSort - : +- CometExchange - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometProject - : : : : : : +- CometBroadcastHashJoin - : : : : : : :- CometProject - : : : : : : : +- CometBroadcastHashJoin - : : : : : : : :- CometProject - : : : : : : : : +- CometBroadcastHashJoin - : : : : : : : : :- CometProject - : : : : : : : : : +- CometBroadcastHashJoin - : : : : : : : : : :- CometFilter - : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_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.inventory - : : : : : : : : +- CometBroadcastExchange - : : : : : : : : +- CometFilter - : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.warehouse - : : : : : : : +- CometBroadcastExchange - : : : : : : : +- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : : : +- CometBroadcastExchange - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.household_demographics - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.promotion - +- CometSort - +- CometExchange - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns - -Comet accelerated 66 out of 68 eligible operators (97%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_2/q73.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_2/q73.native_datafusion/extended.txt deleted file mode 100644 index 4c0ef9abcd..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_2/q73.native_datafusion/extended.txt +++ /dev/null @@ -1,40 +0,0 @@ -CometNativeColumnarToRow -+- CometSort - +- CometExchange - +- CometProject - +- CometBroadcastHashJoin - :- CometFilter - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : : +- CometSubqueryBroadcast - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.store - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.household_demographics - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometNativeScan parquet spark_catalog.default.customer - -Comet accelerated 36 out of 37 eligible operators (97%). Final plan contains 1 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_2/q73.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_2/q73.native_iceberg_compat/extended.txt deleted file mode 100644 index 18ed3e6335..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_2/q73.native_iceberg_compat/extended.txt +++ /dev/null @@ -1,41 +0,0 @@ -CometNativeColumnarToRow -+- CometSort - +- CometExchange - +- CometProject - +- CometBroadcastHashJoin - :- CometFilter - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- 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.store - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.household_demographics - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer - -Comet accelerated 35 out of 37 eligible operators (94%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_2/q74.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_2/q74.native_datafusion/extended.txt deleted file mode 100644 index cfe634568f..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_2/q74.native_datafusion/extended.txt +++ /dev/null @@ -1,88 +0,0 @@ -CometNativeColumnarToRow -+- CometTakeOrderedAndProject - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometHashAggregate - : : : +- CometExchange - : : : +- CometHashAggregate - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.customer - : : : : +- CometBroadcastExchange - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : : +- CometSubqueryBroadcast - : : : : +- CometBroadcastExchange - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometHashAggregate - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.customer - : : : +- CometBroadcastExchange - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : +- CometSubqueryBroadcast - : : : +- CometBroadcastExchange - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometFilter - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.customer - : : +- CometBroadcastExchange - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : +- ReusedSubquery - : +- CometBroadcastExchange - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.date_dim - +- CometBroadcastExchange - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.customer - : +- CometBroadcastExchange - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.web_sales - : +- ReusedSubquery - +- CometBroadcastExchange - +- CometFilter - +- CometNativeScan parquet spark_catalog.default.date_dim - -Comet accelerated 81 out of 85 eligible operators (95%). Final plan contains 1 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_2/q74.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_2/q74.native_iceberg_compat/extended.txt deleted file mode 100644 index 922864baa1..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_2/q74.native_iceberg_compat/extended.txt +++ /dev/null @@ -1,90 +0,0 @@ -CometNativeColumnarToRow -+- CometTakeOrderedAndProject - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometHashAggregate - : : : +- CometExchange - : : : +- CometHashAggregate - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer - : : : : +- CometBroadcastExchange - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometHashAggregate - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer - : : : +- CometBroadcastExchange - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometFilter - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer - : : +- CometBroadcastExchange - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : +- ReusedSubquery - : +- CometBroadcastExchange - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - +- CometBroadcastExchange - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer - : +- CometBroadcastExchange - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : +- ReusedSubquery - +- CometBroadcastExchange - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - -Comet accelerated 79 out of 85 eligible operators (92%). Final plan contains 3 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_2/q75.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_2/q75.native_datafusion/extended.txt deleted file mode 100644 index bccd23b141..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_2/q75.native_datafusion/extended.txt +++ /dev/null @@ -1,170 +0,0 @@ -CometNativeColumnarToRow -+- CometTakeOrderedAndProject - +- CometProject - +- CometSortMergeJoin - :- CometSort - : +- CometExchange - : +- CometFilter - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometUnion - : :- CometProject - : : +- CometSortMergeJoin - : : :- CometSort - : : : +- CometExchange - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : : +- CometSubqueryBroadcast - : : : : : +- CometBroadcastExchange - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : +- CometBroadcastExchange - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- CometSort - : : +- CometExchange - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.catalog_returns - : :- CometProject - : : +- CometSortMergeJoin - : : :- CometSort - : : : +- CometExchange - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : : : +- ReusedSubquery - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : +- CometBroadcastExchange - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- CometSort - : : +- CometExchange - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.store_returns - : +- CometProject - : +- CometSortMergeJoin - : :- CometSort - : : +- CometExchange - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : : +- ReusedSubquery - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.item - : : +- CometBroadcastExchange - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- CometSort - : +- CometExchange - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.web_returns - +- CometSort - +- CometExchange - +- CometFilter - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometUnion - :- CometProject - : +- CometSortMergeJoin - : :- CometSort - : : +- CometExchange - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : +- CometSubqueryBroadcast - : : : : +- CometBroadcastExchange - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.item - : : +- CometBroadcastExchange - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- CometSort - : +- CometExchange - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.catalog_returns - :- CometProject - : +- CometSortMergeJoin - : :- CometSort - : : +- CometExchange - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : : +- ReusedSubquery - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.item - : : +- CometBroadcastExchange - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- CometSort - : +- CometExchange - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.store_returns - +- CometProject - +- CometSortMergeJoin - :- CometSort - : +- CometExchange - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : +- ReusedSubquery - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.item - : +- CometBroadcastExchange - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.date_dim - +- CometSort - +- CometExchange - +- CometProject - +- CometFilter - +- CometNativeScan parquet spark_catalog.default.web_returns - -Comet accelerated 161 out of 167 eligible operators (96%). Final plan contains 1 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_2/q75.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_2/q75.native_iceberg_compat/extended.txt deleted file mode 100644 index ec53af4809..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_2/q75.native_iceberg_compat/extended.txt +++ /dev/null @@ -1,172 +0,0 @@ -CometNativeColumnarToRow -+- CometTakeOrderedAndProject - +- CometProject - +- CometSortMergeJoin - :- CometSort - : +- CometExchange - : +- CometFilter - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometUnion - : :- CometProject - : : +- CometSortMergeJoin - : : :- CometSort - : : : +- CometExchange - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : +- CometBroadcastExchange - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometSort - : : +- CometExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns - : :- CometProject - : : +- CometSortMergeJoin - : : :- CometSort - : : : +- CometExchange - : : : +- 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.item - : : : +- CometBroadcastExchange - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometSort - : : +- CometExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns - : +- CometProject - : +- CometSortMergeJoin - : :- CometSort - : : +- CometExchange - : : +- 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.item - : : +- CometBroadcastExchange - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometSort - : +- CometExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns - +- CometSort - +- CometExchange - +- CometFilter - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometUnion - :- CometProject - : +- CometSortMergeJoin - : :- CometSort - : : +- CometExchange - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : +- CometBroadcastExchange - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometSort - : +- CometExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns - :- CometProject - : +- CometSortMergeJoin - : :- CometSort - : : +- CometExchange - : : +- 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.item - : : +- CometBroadcastExchange - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometSort - : +- CometExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns - +- CometProject - +- CometSortMergeJoin - :- CometSort - : +- CometExchange - : +- 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.item - : +- CometBroadcastExchange - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - +- CometSort - +- CometExchange - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns - -Comet accelerated 159 out of 167 eligible operators (95%). 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_2/q76.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_2/q76.native_datafusion/extended.txt deleted file mode 100644 index cfa19a28b0..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_2/q76.native_datafusion/extended.txt +++ /dev/null @@ -1,47 +0,0 @@ -CometNativeColumnarToRow -+- CometTakeOrderedAndProject - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometUnion - :- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.item - : +- CometBroadcastExchange - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.date_dim - :- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.item - : +- CometBroadcastExchange - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.date_dim - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.item - +- CometBroadcastExchange - +- CometFilter - +- CometNativeScan parquet spark_catalog.default.date_dim - -Comet accelerated 44 out of 44 eligible operators (100%). Final plan contains 1 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_2/q76.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_2/q76.native_iceberg_compat/extended.txt deleted file mode 100644 index 685ff36fed..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_2/q76.native_iceberg_compat/extended.txt +++ /dev/null @@ -1,47 +0,0 @@ -CometNativeColumnarToRow -+- CometTakeOrderedAndProject - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometUnion - :- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : +- CometBroadcastExchange - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - :- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : +- CometBroadcastExchange - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - +- CometBroadcastExchange - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - -Comet accelerated 44 out of 44 eligible operators (100%). Final plan contains 1 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_2/q77.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_2/q77.native_datafusion/extended.txt deleted file mode 100644 index 72280ab0a2..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_2/q77.native_datafusion/extended.txt +++ /dev/null @@ -1,115 +0,0 @@ -TakeOrderedAndProject -+- HashAggregate - +- Exchange - +- HashAggregate - +- Expand - +- Union - :- CometNativeColumnarToRow - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometHashAggregate - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : : +- CometSubqueryBroadcast - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.store - : +- CometBroadcastExchange - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.store_returns - : : : +- ReusedSubquery - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.store - :- Project - : +- BroadcastNestedLoopJoin [COMET: BroadcastNestedLoopJoin is not supported] - : :- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometHashAggregate - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : +- ReusedSubquery - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- CometNativeColumnarToRow - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometNativeScan parquet spark_catalog.default.catalog_returns - : : +- ReusedSubquery - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.date_dim - +- CometNativeColumnarToRow - +- CometProject - +- CometBroadcastHashJoin - :- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : +- ReusedSubquery - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.web_page - +- CometBroadcastExchange - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.web_returns - : : +- ReusedSubquery - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.date_dim - +- CometBroadcastExchange - +- CometFilter - +- CometNativeScan parquet spark_catalog.default.web_page - -Comet accelerated 94 out of 109 eligible operators (86%). Final plan contains 4 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_2/q77.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_2/q77.native_iceberg_compat/extended.txt deleted file mode 100644 index b2653c7068..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_2/q77.native_iceberg_compat/extended.txt +++ /dev/null @@ -1,116 +0,0 @@ -TakeOrderedAndProject -+- HashAggregate - +- 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 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_2/q78.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_2/q78.native_datafusion/extended.txt deleted file mode 100644 index d29dbc13e5..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_2/q78.native_datafusion/extended.txt +++ /dev/null @@ -1,79 +0,0 @@ -TakeOrderedAndProject -+- Project [COMET: Comet does not support Spark's BigDecimal rounding] - +- CometNativeColumnarToRow - +- CometSortMergeJoin - :- CometProject - : +- CometSortMergeJoin - : :- CometSort - : : +- CometHashAggregate - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometFilter - : : : +- CometSortMergeJoin - : : : :- CometSort - : : : : +- CometExchange - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : : +- CometSubqueryBroadcast - : : : : +- CometBroadcastExchange - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- CometSort - : : : +- CometExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.store_returns - : : +- CometBroadcastExchange - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- CometSort - : +- CometFilter - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometFilter - : : +- CometSortMergeJoin - : : :- CometSort - : : : +- CometExchange - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : +- ReusedSubquery - : : +- CometSort - : : +- CometExchange - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.web_returns - : +- CometBroadcastExchange - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.date_dim - +- CometSort - +- CometFilter - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometFilter - : +- CometSortMergeJoin - : :- CometSort - : : +- CometExchange - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : +- ReusedSubquery - : +- CometSort - : +- CometExchange - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.catalog_returns - +- CometBroadcastExchange - +- CometFilter - +- CometNativeScan parquet spark_catalog.default.date_dim - -Comet accelerated 71 out of 76 eligible operators (93%). Final plan contains 1 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_2/q78.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_2/q78.native_iceberg_compat/extended.txt deleted file mode 100644 index 1b1e6d0cde..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_2/q78.native_iceberg_compat/extended.txt +++ /dev/null @@ -1,80 +0,0 @@ -TakeOrderedAndProject -+- Project [COMET: Comet does not support Spark's BigDecimal rounding] - +- CometNativeColumnarToRow - +- CometSortMergeJoin - :- CometProject - : +- CometSortMergeJoin - : :- CometSort - : : +- CometHashAggregate - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometFilter - : : : +- CometSortMergeJoin - : : : :- CometSort - : : : : +- CometExchange - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometSort - : : : +- CometExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns - : : +- CometBroadcastExchange - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometSort - : +- CometFilter - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometFilter - : : +- CometSortMergeJoin - : : :- CometSort - : : : +- CometExchange - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : +- ReusedSubquery - : : +- CometSort - : : +- CometExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns - : +- CometBroadcastExchange - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - +- CometSort - +- CometFilter - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometFilter - : +- CometSortMergeJoin - : :- CometSort - : : +- CometExchange - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : +- ReusedSubquery - : +- CometSort - : +- CometExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns - +- CometBroadcastExchange - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - -Comet accelerated 70 out of 76 eligible operators (92%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_2/q79.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_2/q79.native_datafusion/extended.txt deleted file mode 100644 index 5af6449428..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_2/q79.native_datafusion/extended.txt +++ /dev/null @@ -1,38 +0,0 @@ -CometNativeColumnarToRow -+- CometTakeOrderedAndProject - +- CometProject - +- CometBroadcastHashJoin - :- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : : +- CometSubqueryBroadcast - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.store - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.household_demographics - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometNativeScan parquet spark_catalog.default.customer - -Comet accelerated 34 out of 35 eligible operators (97%). Final plan contains 1 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_2/q79.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_2/q79.native_iceberg_compat/extended.txt deleted file mode 100644 index f3e80e01cb..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_2/q79.native_iceberg_compat/extended.txt +++ /dev/null @@ -1,39 +0,0 @@ -CometNativeColumnarToRow -+- CometTakeOrderedAndProject - +- CometProject - +- CometBroadcastHashJoin - :- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- 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.store - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.household_demographics - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer - -Comet accelerated 33 out of 35 eligible operators (94%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_2/q8.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_2/q8.native_datafusion/extended.txt deleted file mode 100644 index 209a4e2bc0..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_2/q8.native_datafusion/extended.txt +++ /dev/null @@ -1,51 +0,0 @@ -CometNativeColumnarToRow -+- CometTakeOrderedAndProject - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : +- CometSubqueryBroadcast - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.store - +- CometBroadcastExchange - +- 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 47 out of 48 eligible operators (97%). Final plan contains 1 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_2/q8.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_2/q8.native_iceberg_compat/extended.txt deleted file mode 100644 index be5abf4913..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_2/q8.native_iceberg_compat/extended.txt +++ /dev/null @@ -1,52 +0,0 @@ -CometNativeColumnarToRow -+- CometTakeOrderedAndProject - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- 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.store - +- CometBroadcastExchange - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometBroadcastHashJoin - :- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer - -Comet accelerated 46 out of 48 eligible operators (95%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_2/q80.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_2/q80.native_datafusion/extended.txt deleted file mode 100644 index 9f67d9b8b5..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_2/q80.native_datafusion/extended.txt +++ /dev/null @@ -1,130 +0,0 @@ -CometNativeColumnarToRow -+- CometTakeOrderedAndProject - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometExpand - +- CometUnion - :- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometSortMergeJoin - : : : : : :- CometSort - : : : : : : +- CometExchange - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : : : : +- CometSubqueryBroadcast - : : : : : : +- CometBroadcastExchange - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : : +- CometSort - : : : : : +- CometExchange - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.store_returns - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.store - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.item - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.promotion - :- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometSortMergeJoin - : : : : : :- CometSort - : : : : : : +- CometExchange - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : : : +- ReusedSubquery - : : : : : +- CometSort - : : : : : +- CometExchange - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_returns - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.catalog_page - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.item - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.promotion - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometSortMergeJoin - : : : : :- CometSort - : : : : : +- CometExchange - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : : : +- ReusedSubquery - : : : : +- CometSort - : : : : +- CometExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.web_returns - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.web_site - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.item - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometNativeScan parquet spark_catalog.default.promotion - -Comet accelerated 124 out of 127 eligible operators (97%). Final plan contains 1 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_2/q80.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_2/q80.native_iceberg_compat/extended.txt deleted file mode 100644 index ed8d0deb6d..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_2/q80.native_iceberg_compat/extended.txt +++ /dev/null @@ -1,131 +0,0 @@ -CometNativeColumnarToRow -+- CometTakeOrderedAndProject - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometExpand - +- CometUnion - :- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometSortMergeJoin - : : : : : :- CometSort - : : : : : : +- CometExchange - : : : : : : +- 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 - : : : : : +- 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.date_dim - : : : +- 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.promotion - :- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometSortMergeJoin - : : : : : :- CometSort - : : : : : : +- CometExchange - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : : +- ReusedSubquery - : : : : : +- CometSort - : : : : : +- CometExchange - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_page - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.promotion - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometSortMergeJoin - : : : : :- CometSort - : : : : : +- CometExchange - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : : +- ReusedSubquery - : : : : +- CometSort - : : : : +- CometExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_site - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.promotion - -Comet accelerated 123 out of 127 eligible operators (96%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_2/q81.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_2/q81.native_datafusion/extended.txt deleted file mode 100644 index f10339641b..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_2/q81.native_datafusion/extended.txt +++ /dev/null @@ -1,64 +0,0 @@ -CometNativeColumnarToRow -+- CometTakeOrderedAndProject - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometHashAggregate - : : : +- CometExchange - : : : +- CometHashAggregate - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_returns - : : : : : +- CometSubqueryBroadcast - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.customer_address - : : +- CometBroadcastExchange - : : +- CometFilter - : : +- CometHashAggregate - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometHashAggregate - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.catalog_returns - : : : : +- ReusedSubquery - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.customer_address - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.customer - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometNativeScan parquet spark_catalog.default.customer_address - -Comet accelerated 59 out of 61 eligible operators (96%). Final plan contains 1 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_2/q81.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_2/q81.native_iceberg_compat/extended.txt deleted file mode 100644 index eb965c592c..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_2/q81.native_iceberg_compat/extended.txt +++ /dev/null @@ -1,65 +0,0 @@ -CometNativeColumnarToRow -+- CometTakeOrderedAndProject - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometHashAggregate - : : : +- CometExchange - : : : +- CometHashAggregate - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns - : : : : : +- 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_address - : : +- CometBroadcastExchange - : : +- CometFilter - : : +- CometHashAggregate - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometHashAggregate - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns - : : : : +- 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_address - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address - -Comet accelerated 58 out of 61 eligible operators (95%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_2/q82.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_2/q82.native_datafusion/extended.txt deleted file mode 100644 index e7b75e3a55..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_2/q82.native_datafusion/extended.txt +++ /dev/null @@ -1,33 +0,0 @@ -CometNativeColumnarToRow -+- CometTakeOrderedAndProject - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometBroadcastExchange - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.item - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.inventory - : : +- CometSubqueryBroadcast - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.date_dim - +- CometProject - +- CometFilter - +- CometNativeScan parquet spark_catalog.default.store_sales - -Comet accelerated 29 out of 30 eligible operators (96%). Final plan contains 1 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_2/q82.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_2/q82.native_iceberg_compat/extended.txt deleted file mode 100644 index 17e0a010c0..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_2/q82.native_iceberg_compat/extended.txt +++ /dev/null @@ -1,34 +0,0 @@ -CometNativeColumnarToRow -+- CometTakeOrderedAndProject - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometBroadcastExchange - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : +- CometBroadcastExchange - : : +- CometProject - : : +- 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 - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - -Comet accelerated 28 out of 30 eligible operators (93%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_2/q83.ansi.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_2/q83.ansi.native_datafusion/extended.txt deleted file mode 100644 index 32adaba671..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_2/q83.ansi.native_datafusion/extended.txt +++ /dev/null @@ -1,104 +0,0 @@ -CometNativeColumnarToRow -+- CometTakeOrderedAndProject - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometHashAggregate - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.store_returns - : : : : +- CometSubqueryBroadcast - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometNativeScan parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.item - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometNativeScan parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.catalog_returns - : : : +- ReusedSubquery - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.item - : +- CometBroadcastExchange - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometNativeScan parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.date_dim - +- CometBroadcastExchange - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.web_returns - : : +- ReusedSubquery - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.item - +- CometBroadcastExchange - +- CometProject - +- CometBroadcastHashJoin - :- CometFilter - : +- CometNativeScan parquet spark_catalog.default.date_dim - +- CometBroadcastExchange - +- CometProject - +- CometBroadcastHashJoin - :- CometNativeScan parquet spark_catalog.default.date_dim - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometNativeScan parquet spark_catalog.default.date_dim - -Comet accelerated 98 out of 101 eligible operators (97%). Final plan contains 1 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_2/q83.ansi.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_2/q83.ansi.native_iceberg_compat/extended.txt deleted file mode 100644 index fac88c8bc7..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_2/q83.ansi.native_iceberg_compat/extended.txt +++ /dev/null @@ -1,105 +0,0 @@ -CometNativeColumnarToRow -+- CometTakeOrderedAndProject - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometHashAggregate - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- 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 - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns - : : : +- ReusedSubquery - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : +- CometBroadcastExchange - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - +- 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.item - +- CometBroadcastExchange - +- CometProject - +- CometBroadcastHashJoin - :- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - +- CometBroadcastExchange - +- CometProject - +- CometBroadcastHashJoin - :- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - -Comet accelerated 97 out of 101 eligible operators (96%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_2/q84.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_2/q84.native_datafusion/extended.txt deleted file mode 100644 index 402322f69f..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_2/q84.native_datafusion/extended.txt +++ /dev/null @@ -1,35 +0,0 @@ -CometNativeColumnarToRow -+- CometTakeOrderedAndProject - +- CometProject - +- CometBroadcastHashJoin - :- CometBroadcastExchange - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.customer - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.customer_address - : : : +- CometBroadcastExchange - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.customer_demographics - : : +- CometBroadcastExchange - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.household_demographics - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.income_band - +- CometProject - +- CometFilter - +- CometNativeScan parquet spark_catalog.default.store_returns - -Comet accelerated 32 out of 32 eligible operators (100%). Final plan contains 1 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_2/q84.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_2/q84.native_iceberg_compat/extended.txt deleted file mode 100644 index fcd8384c22..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_2/q84.native_iceberg_compat/extended.txt +++ /dev/null @@ -1,35 +0,0 @@ -CometNativeColumnarToRow -+- CometTakeOrderedAndProject - +- CometProject - +- CometBroadcastHashJoin - :- CometBroadcastExchange - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address - : : : +- CometBroadcastExchange - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics - : : +- CometBroadcastExchange - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.household_demographics - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.income_band - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns - -Comet accelerated 32 out of 32 eligible operators (100%). Final plan contains 1 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_2/q85.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_2/q85.native_datafusion/extended.txt deleted file mode 100644 index 13506e196f..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_2/q85.native_datafusion/extended.txt +++ /dev/null @@ -1,55 +0,0 @@ -CometNativeColumnarToRow -+- CometTakeOrderedAndProject - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometProject - : : : : : : +- CometBroadcastHashJoin - : : : : : : :- CometBroadcastExchange - : : : : : : : +- CometFilter - : : : : : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : : : : : +- CometSubqueryBroadcast - : : : : : : : +- CometBroadcastExchange - : : : : : : : +- CometProject - : : : : : : : +- CometFilter - : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.web_returns - : : : : : +- CometBroadcastExchange - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.web_page - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.customer_demographics - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.customer_demographics - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.customer_address - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.date_dim - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometNativeScan parquet spark_catalog.default.reason - -Comet accelerated 51 out of 52 eligible operators (98%). Final plan contains 1 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_2/q85.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_2/q85.native_iceberg_compat/extended.txt deleted file mode 100644 index f99808facc..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_2/q85.native_iceberg_compat/extended.txt +++ /dev/null @@ -1,56 +0,0 @@ -CometNativeColumnarToRow -+- CometTakeOrderedAndProject - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometProject - : : : : : : +- CometBroadcastHashJoin - : : : : : : :- CometBroadcastExchange - : : : : : : : +- 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 - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns - : : : : : +- CometBroadcastExchange - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_page - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics - : : +- 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.reason - -Comet accelerated 50 out of 52 eligible operators (96%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_2/q86.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_2/q86.native_datafusion/extended.txt deleted file mode 100644 index bddf80b4d0..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_2/q86.native_datafusion/extended.txt +++ /dev/null @@ -1,31 +0,0 @@ -TakeOrderedAndProject -+- 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 - +- CometExpand - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : +- CometSubqueryBroadcast - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.date_dim - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometNativeScan parquet spark_catalog.default.item - -Comet accelerated 24 out of 28 eligible operators (85%). Final plan contains 1 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_2/q86.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_2/q86.native_iceberg_compat/extended.txt deleted file mode 100644 index c0056e2382..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_2/q86.native_iceberg_compat/extended.txt +++ /dev/null @@ -1,32 +0,0 @@ -TakeOrderedAndProject -+- 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 - +- CometExpand - +- 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 - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - -Comet accelerated 23 out of 28 eligible operators (82%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_2/q87.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_2/q87.native_datafusion/extended.txt deleted file mode 100644 index c965be9cc3..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_2/q87.native_datafusion/extended.txt +++ /dev/null @@ -1,69 +0,0 @@ -CometNativeColumnarToRow -+- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometBroadcastHashJoin - : :- CometHashAggregate - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : : +- CometSubqueryBroadcast - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.customer - : +- CometBroadcastExchange - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : +- ReusedSubquery - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.customer - +- CometBroadcastExchange - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : +- ReusedSubquery - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.date_dim - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometNativeScan parquet spark_catalog.default.customer - -Comet accelerated 63 out of 66 eligible operators (95%). Final plan contains 1 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_2/q87.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_2/q87.native_iceberg_compat/extended.txt deleted file mode 100644 index cb4d06350b..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_2/q87.native_iceberg_compat/extended.txt +++ /dev/null @@ -1,70 +0,0 @@ -CometNativeColumnarToRow -+- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- 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 - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer - : +- CometBroadcastExchange - : +- 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 - +- CometBroadcastExchange - +- 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 62 out of 66 eligible operators (93%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_2/q88.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_2/q88.native_datafusion/extended.txt deleted file mode 100644 index 87f19deeb8..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_2/q88.native_datafusion/extended.txt +++ /dev/null @@ -1,216 +0,0 @@ -BroadcastNestedLoopJoin -:- BroadcastNestedLoopJoin -: :- BroadcastNestedLoopJoin -: : :- BroadcastNestedLoopJoin -: : : :- BroadcastNestedLoopJoin -: : : : :- BroadcastNestedLoopJoin -: : : : : :- BroadcastNestedLoopJoin [COMET: BroadcastNestedLoopJoin is not supported] -: : : : : : :- CometNativeColumnarToRow -: : : : : : : +- CometHashAggregate -: : : : : : : +- CometExchange -: : : : : : : +- CometHashAggregate -: : : : : : : +- CometProject -: : : : : : : +- CometBroadcastHashJoin -: : : : : : : :- CometProject -: : : : : : : : +- CometBroadcastHashJoin -: : : : : : : : :- CometProject -: : : : : : : : : +- CometBroadcastHashJoin -: : : : : : : : : :- CometProject -: : : : : : : : : : +- CometFilter -: : : : : : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales -: : : : : : : : : +- CometBroadcastExchange -: : : : : : : : : +- CometProject -: : : : : : : : : +- CometFilter -: : : : : : : : : +- CometNativeScan parquet spark_catalog.default.household_demographics -: : : : : : : : +- CometBroadcastExchange -: : : : : : : : +- CometProject -: : : : : : : : +- CometFilter -: : : : : : : : +- CometNativeScan parquet spark_catalog.default.time_dim -: : : : : : : +- CometBroadcastExchange -: : : : : : : +- CometProject -: : : : : : : +- CometFilter -: : : : : : : +- CometNativeScan parquet spark_catalog.default.store -: : : : : : +- BroadcastExchange -: : : : : : +- CometNativeColumnarToRow -: : : : : : +- CometHashAggregate -: : : : : : +- CometExchange -: : : : : : +- CometHashAggregate -: : : : : : +- CometProject -: : : : : : +- CometBroadcastHashJoin -: : : : : : :- CometProject -: : : : : : : +- CometBroadcastHashJoin -: : : : : : : :- CometProject -: : : : : : : : +- CometBroadcastHashJoin -: : : : : : : : :- CometProject -: : : : : : : : : +- CometFilter -: : : : : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales -: : : : : : : : +- CometBroadcastExchange -: : : : : : : : +- CometProject -: : : : : : : : +- CometFilter -: : : : : : : : +- CometNativeScan parquet spark_catalog.default.household_demographics -: : : : : : : +- CometBroadcastExchange -: : : : : : : +- CometProject -: : : : : : : +- CometFilter -: : : : : : : +- CometNativeScan parquet spark_catalog.default.time_dim -: : : : : : +- CometBroadcastExchange -: : : : : : +- CometProject -: : : : : : +- CometFilter -: : : : : : +- CometNativeScan parquet spark_catalog.default.store -: : : : : +- BroadcastExchange -: : : : : +- CometNativeColumnarToRow -: : : : : +- CometHashAggregate -: : : : : +- CometExchange -: : : : : +- CometHashAggregate -: : : : : +- CometProject -: : : : : +- CometBroadcastHashJoin -: : : : : :- CometProject -: : : : : : +- CometBroadcastHashJoin -: : : : : : :- CometProject -: : : : : : : +- CometBroadcastHashJoin -: : : : : : : :- CometProject -: : : : : : : : +- CometFilter -: : : : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales -: : : : : : : +- CometBroadcastExchange -: : : : : : : +- CometProject -: : : : : : : +- CometFilter -: : : : : : : +- CometNativeScan parquet spark_catalog.default.household_demographics -: : : : : : +- CometBroadcastExchange -: : : : : : +- CometProject -: : : : : : +- CometFilter -: : : : : : +- CometNativeScan parquet spark_catalog.default.time_dim -: : : : : +- CometBroadcastExchange -: : : : : +- CometProject -: : : : : +- CometFilter -: : : : : +- CometNativeScan parquet spark_catalog.default.store -: : : : +- BroadcastExchange -: : : : +- CometNativeColumnarToRow -: : : : +- CometHashAggregate -: : : : +- CometExchange -: : : : +- CometHashAggregate -: : : : +- CometProject -: : : : +- CometBroadcastHashJoin -: : : : :- CometProject -: : : : : +- CometBroadcastHashJoin -: : : : : :- CometProject -: : : : : : +- CometBroadcastHashJoin -: : : : : : :- CometProject -: : : : : : : +- CometFilter -: : : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales -: : : : : : +- CometBroadcastExchange -: : : : : : +- CometProject -: : : : : : +- CometFilter -: : : : : : +- CometNativeScan parquet spark_catalog.default.household_demographics -: : : : : +- CometBroadcastExchange -: : : : : +- CometProject -: : : : : +- CometFilter -: : : : : +- CometNativeScan parquet spark_catalog.default.time_dim -: : : : +- CometBroadcastExchange -: : : : +- CometProject -: : : : +- CometFilter -: : : : +- CometNativeScan parquet spark_catalog.default.store -: : : +- BroadcastExchange -: : : +- CometNativeColumnarToRow -: : : +- CometHashAggregate -: : : +- CometExchange -: : : +- CometHashAggregate -: : : +- CometProject -: : : +- CometBroadcastHashJoin -: : : :- CometProject -: : : : +- CometBroadcastHashJoin -: : : : :- CometProject -: : : : : +- CometBroadcastHashJoin -: : : : : :- CometProject -: : : : : : +- CometFilter -: : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales -: : : : : +- CometBroadcastExchange -: : : : : +- CometProject -: : : : : +- CometFilter -: : : : : +- CometNativeScan parquet spark_catalog.default.household_demographics -: : : : +- CometBroadcastExchange -: : : : +- CometProject -: : : : +- CometFilter -: : : : +- CometNativeScan parquet spark_catalog.default.time_dim -: : : +- CometBroadcastExchange -: : : +- CometProject -: : : +- CometFilter -: : : +- CometNativeScan parquet spark_catalog.default.store -: : +- BroadcastExchange -: : +- CometNativeColumnarToRow -: : +- CometHashAggregate -: : +- CometExchange -: : +- CometHashAggregate -: : +- CometProject -: : +- CometBroadcastHashJoin -: : :- CometProject -: : : +- CometBroadcastHashJoin -: : : :- CometProject -: : : : +- CometBroadcastHashJoin -: : : : :- CometProject -: : : : : +- CometFilter -: : : : : +- CometNativeScan parquet spark_catalog.default.store_sales -: : : : +- CometBroadcastExchange -: : : : +- CometProject -: : : : +- CometFilter -: : : : +- CometNativeScan parquet spark_catalog.default.household_demographics -: : : +- CometBroadcastExchange -: : : +- CometProject -: : : +- CometFilter -: : : +- CometNativeScan parquet spark_catalog.default.time_dim -: : +- CometBroadcastExchange -: : +- CometProject -: : +- CometFilter -: : +- CometNativeScan parquet spark_catalog.default.store -: +- BroadcastExchange -: +- CometNativeColumnarToRow -: +- CometHashAggregate -: +- CometExchange -: +- CometHashAggregate -: +- CometProject -: +- CometBroadcastHashJoin -: :- CometProject -: : +- CometBroadcastHashJoin -: : :- CometProject -: : : +- CometBroadcastHashJoin -: : : :- CometProject -: : : : +- CometFilter -: : : : +- CometNativeScan parquet spark_catalog.default.store_sales -: : : +- CometBroadcastExchange -: : : +- CometProject -: : : +- CometFilter -: : : +- CometNativeScan parquet spark_catalog.default.household_demographics -: : +- CometBroadcastExchange -: : +- CometProject -: : +- CometFilter -: : +- CometNativeScan parquet spark_catalog.default.time_dim -: +- CometBroadcastExchange -: +- CometProject -: +- CometFilter -: +- CometNativeScan parquet spark_catalog.default.store -+- BroadcastExchange - +- CometNativeColumnarToRow - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.household_demographics - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.time_dim - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometNativeScan parquet spark_catalog.default.store - -Comet accelerated 192 out of 206 eligible operators (93%). Final plan contains 8 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_2/q88.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_2/q88.native_iceberg_compat/extended.txt deleted file mode 100644 index f1e4c8679a..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_2/q88.native_iceberg_compat/extended.txt +++ /dev/null @@ -1,216 +0,0 @@ -BroadcastNestedLoopJoin -:- BroadcastNestedLoopJoin -: :- BroadcastNestedLoopJoin -: : :- BroadcastNestedLoopJoin -: : : :- BroadcastNestedLoopJoin -: : : : :- BroadcastNestedLoopJoin -: : : : : :- BroadcastNestedLoopJoin [COMET: BroadcastNestedLoopJoin is not supported] -: : : : : : :- CometNativeColumnarToRow -: : : : : : : +- CometHashAggregate -: : : : : : : +- CometExchange -: : : : : : : +- CometHashAggregate -: : : : : : : +- CometProject -: : : : : : : +- CometBroadcastHashJoin -: : : : : : : :- CometProject -: : : : : : : : +- CometBroadcastHashJoin -: : : : : : : : :- CometProject -: : : : : : : : : +- CometBroadcastHashJoin -: : : : : : : : : :- CometProject -: : : : : : : : : : +- CometFilter -: : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -: : : : : : : : : +- CometBroadcastExchange -: : : : : : : : : +- CometProject -: : : : : : : : : +- CometFilter -: : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.household_demographics -: : : : : : : : +- CometBroadcastExchange -: : : : : : : : +- CometProject -: : : : : : : : +- CometFilter -: : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.time_dim -: : : : : : : +- CometBroadcastExchange -: : : : : : : +- CometProject -: : : : : : : +- CometFilter -: : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store -: : : : : : +- BroadcastExchange -: : : : : : +- CometNativeColumnarToRow -: : : : : : +- CometHashAggregate -: : : : : : +- CometExchange -: : : : : : +- CometHashAggregate -: : : : : : +- CometProject -: : : : : : +- CometBroadcastHashJoin -: : : : : : :- CometProject -: : : : : : : +- CometBroadcastHashJoin -: : : : : : : :- CometProject -: : : : : : : : +- CometBroadcastHashJoin -: : : : : : : : :- CometProject -: : : : : : : : : +- CometFilter -: : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -: : : : : : : : +- CometBroadcastExchange -: : : : : : : : +- CometProject -: : : : : : : : +- CometFilter -: : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.household_demographics -: : : : : : : +- CometBroadcastExchange -: : : : : : : +- CometProject -: : : : : : : +- CometFilter -: : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.time_dim -: : : : : : +- CometBroadcastExchange -: : : : : : +- CometProject -: : : : : : +- CometFilter -: : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store -: : : : : +- BroadcastExchange -: : : : : +- CometNativeColumnarToRow -: : : : : +- CometHashAggregate -: : : : : +- CometExchange -: : : : : +- CometHashAggregate -: : : : : +- CometProject -: : : : : +- CometBroadcastHashJoin -: : : : : :- CometProject -: : : : : : +- CometBroadcastHashJoin -: : : : : : :- CometProject -: : : : : : : +- CometBroadcastHashJoin -: : : : : : : :- CometProject -: : : : : : : : +- CometFilter -: : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -: : : : : : : +- CometBroadcastExchange -: : : : : : : +- CometProject -: : : : : : : +- CometFilter -: : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.household_demographics -: : : : : : +- CometBroadcastExchange -: : : : : : +- CometProject -: : : : : : +- CometFilter -: : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.time_dim -: : : : : +- CometBroadcastExchange -: : : : : +- CometProject -: : : : : +- CometFilter -: : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store -: : : : +- BroadcastExchange -: : : : +- CometNativeColumnarToRow -: : : : +- CometHashAggregate -: : : : +- CometExchange -: : : : +- CometHashAggregate -: : : : +- CometProject -: : : : +- CometBroadcastHashJoin -: : : : :- CometProject -: : : : : +- CometBroadcastHashJoin -: : : : : :- CometProject -: : : : : : +- CometBroadcastHashJoin -: : : : : : :- CometProject -: : : : : : : +- CometFilter -: : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -: : : : : : +- CometBroadcastExchange -: : : : : : +- CometProject -: : : : : : +- CometFilter -: : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.household_demographics -: : : : : +- CometBroadcastExchange -: : : : : +- CometProject -: : : : : +- CometFilter -: : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.time_dim -: : : : +- CometBroadcastExchange -: : : : +- CometProject -: : : : +- CometFilter -: : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store -: : : +- BroadcastExchange -: : : +- CometNativeColumnarToRow -: : : +- CometHashAggregate -: : : +- CometExchange -: : : +- CometHashAggregate -: : : +- CometProject -: : : +- CometBroadcastHashJoin -: : : :- CometProject -: : : : +- CometBroadcastHashJoin -: : : : :- CometProject -: : : : : +- CometBroadcastHashJoin -: : : : : :- CometProject -: : : : : : +- CometFilter -: : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -: : : : : +- CometBroadcastExchange -: : : : : +- CometProject -: : : : : +- CometFilter -: : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.household_demographics -: : : : +- CometBroadcastExchange -: : : : +- CometProject -: : : : +- CometFilter -: : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.time_dim -: : : +- CometBroadcastExchange -: : : +- CometProject -: : : +- CometFilter -: : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store -: : +- BroadcastExchange -: : +- CometNativeColumnarToRow -: : +- CometHashAggregate -: : +- CometExchange -: : +- CometHashAggregate -: : +- CometProject -: : +- CometBroadcastHashJoin -: : :- CometProject -: : : +- CometBroadcastHashJoin -: : : :- CometProject -: : : : +- CometBroadcastHashJoin -: : : : :- CometProject -: : : : : +- CometFilter -: : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -: : : : +- CometBroadcastExchange -: : : : +- CometProject -: : : : +- CometFilter -: : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.household_demographics -: : : +- CometBroadcastExchange -: : : +- CometProject -: : : +- CometFilter -: : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.time_dim -: : +- CometBroadcastExchange -: : +- CometProject -: : +- CometFilter -: : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store -: +- BroadcastExchange -: +- CometNativeColumnarToRow -: +- CometHashAggregate -: +- CometExchange -: +- CometHashAggregate -: +- CometProject -: +- CometBroadcastHashJoin -: :- CometProject -: : +- CometBroadcastHashJoin -: : :- CometProject -: : : +- CometBroadcastHashJoin -: : : :- CometProject -: : : : +- CometFilter -: : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -: : : +- CometBroadcastExchange -: : : +- CometProject -: : : +- CometFilter -: : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.household_demographics -: : +- CometBroadcastExchange -: : +- CometProject -: : +- CometFilter -: : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.time_dim -: +- CometBroadcastExchange -: +- CometProject -: +- CometFilter -: +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store -+- BroadcastExchange - +- CometNativeColumnarToRow - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.household_demographics - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.time_dim - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - -Comet accelerated 192 out of 206 eligible operators (93%). Final plan contains 8 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_2/q89.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_2/q89.native_datafusion/extended.txt deleted file mode 100644 index 9a8a9bbba4..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_2/q89.native_datafusion/extended.txt +++ /dev/null @@ -1,36 +0,0 @@ -TakeOrderedAndProject -+- 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 - +- CometExchange - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.item - : : +- CometBroadcastExchange - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : +- CometSubqueryBroadcast - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.date_dim - +- CometBroadcastExchange - +- CometFilter - +- CometNativeScan parquet spark_catalog.default.store - -Comet accelerated 28 out of 33 eligible operators (84%). Final plan contains 1 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_2/q89.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_2/q89.native_iceberg_compat/extended.txt deleted file mode 100644 index 093505a6cc..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_2/q89.native_iceberg_compat/extended.txt +++ /dev/null @@ -1,37 +0,0 @@ -TakeOrderedAndProject -+- 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 - +- CometExchange - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : +- CometBroadcastExchange - : : +- 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 - -Comet accelerated 27 out of 33 eligible operators (81%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_2/q9.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_2/q9.native_datafusion/extended.txt deleted file mode 100644 index ff39852ed0..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_2/q9.native_datafusion/extended.txt +++ /dev/null @@ -1,61 +0,0 @@ - Project [COMET: ] -: :- Subquery -: : +- CometNativeColumnarToRow -: : +- CometProject -: : +- CometHashAggregate -: : +- CometExchange -: : +- CometHashAggregate -: : +- CometProject -: : +- CometFilter -: : +- CometNativeScan parquet spark_catalog.default.store_sales -: :- ReusedSubquery -: :- ReusedSubquery -: :- Subquery -: : +- CometNativeColumnarToRow -: : +- CometProject -: : +- CometHashAggregate -: : +- CometExchange -: : +- CometHashAggregate -: : +- CometProject -: : +- CometFilter -: : +- CometNativeScan parquet spark_catalog.default.store_sales -: :- ReusedSubquery -: :- ReusedSubquery -: :- Subquery -: : +- CometNativeColumnarToRow -: : +- CometProject -: : +- CometHashAggregate -: : +- CometExchange -: : +- CometHashAggregate -: : +- CometProject -: : +- CometFilter -: : +- CometNativeScan parquet spark_catalog.default.store_sales -: :- ReusedSubquery -: :- ReusedSubquery -: :- Subquery -: : +- CometNativeColumnarToRow -: : +- CometProject -: : +- CometHashAggregate -: : +- CometExchange -: : +- CometHashAggregate -: : +- CometProject -: : +- CometFilter -: : +- CometNativeScan parquet spark_catalog.default.store_sales -: :- ReusedSubquery -: :- ReusedSubquery -: :- Subquery -: : +- CometNativeColumnarToRow -: : +- CometProject -: : +- CometHashAggregate -: : +- CometExchange -: : +- CometHashAggregate -: : +- CometProject -: : +- CometFilter -: : +- CometNativeScan parquet spark_catalog.default.store_sales -: :- ReusedSubquery -: +- ReusedSubquery -+- CometNativeColumnarToRow - +- CometFilter - +- CometNativeScan parquet spark_catalog.default.reason - -Comet accelerated 37 out of 53 eligible operators (69%). 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_2/q9.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_2/q9.native_iceberg_compat/extended.txt deleted file mode 100644 index e3614131fe..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_2/q9.native_iceberg_compat/extended.txt +++ /dev/null @@ -1,61 +0,0 @@ - Project [COMET: ] -: :- Subquery -: : +- CometNativeColumnarToRow -: : +- CometProject -: : +- CometHashAggregate -: : +- CometExchange -: : +- CometHashAggregate -: : +- CometProject -: : +- CometFilter -: : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -: :- ReusedSubquery -: :- ReusedSubquery -: :- Subquery -: : +- CometNativeColumnarToRow -: : +- CometProject -: : +- CometHashAggregate -: : +- CometExchange -: : +- CometHashAggregate -: : +- CometProject -: : +- CometFilter -: : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -: :- ReusedSubquery -: :- ReusedSubquery -: :- Subquery -: : +- CometNativeColumnarToRow -: : +- CometProject -: : +- CometHashAggregate -: : +- CometExchange -: : +- CometHashAggregate -: : +- CometProject -: : +- CometFilter -: : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -: :- ReusedSubquery -: :- ReusedSubquery -: :- Subquery -: : +- CometNativeColumnarToRow -: : +- CometProject -: : +- CometHashAggregate -: : +- CometExchange -: : +- CometHashAggregate -: : +- CometProject -: : +- CometFilter -: : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -: :- ReusedSubquery -: :- ReusedSubquery -: :- Subquery -: : +- CometNativeColumnarToRow -: : +- CometProject -: : +- CometHashAggregate -: : +- CometExchange -: : +- CometHashAggregate -: : +- CometProject -: : +- CometFilter -: : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -: :- ReusedSubquery -: +- ReusedSubquery -+- CometNativeColumnarToRow - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.reason - -Comet accelerated 37 out of 53 eligible operators (69%). 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_2/q90.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_2/q90.native_datafusion/extended.txt deleted file mode 100644 index b1125e15e4..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_2/q90.native_datafusion/extended.txt +++ /dev/null @@ -1,55 +0,0 @@ -Project -+- BroadcastNestedLoopJoin [COMET: BroadcastNestedLoopJoin is not supported] - :- CometNativeColumnarToRow - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.household_demographics - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.time_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.web_page - +- BroadcastExchange - +- CometNativeColumnarToRow - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.household_demographics - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.time_dim - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometNativeScan parquet spark_catalog.default.web_page - -Comet accelerated 48 out of 51 eligible operators (94%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_2/q90.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_2/q90.native_iceberg_compat/extended.txt deleted file mode 100644 index 3b1bd00423..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_2/q90.native_iceberg_compat/extended.txt +++ /dev/null @@ -1,55 +0,0 @@ -Project -+- BroadcastNestedLoopJoin [COMET: BroadcastNestedLoopJoin is not supported] - :- CometNativeColumnarToRow - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.household_demographics - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.time_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_page - +- BroadcastExchange - +- CometNativeColumnarToRow - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.household_demographics - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.time_dim - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_page - -Comet accelerated 48 out of 51 eligible operators (94%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_2/q91.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_2/q91.native_datafusion/extended.txt deleted file mode 100644 index 9b191f8492..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_2/q91.native_datafusion/extended.txt +++ /dev/null @@ -1,50 +0,0 @@ -CometNativeColumnarToRow -+- CometSort - +- CometExchange - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.call_center - : : : : : +- CometBroadcastExchange - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_returns - : : : : : +- CometSubqueryBroadcast - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.customer - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.customer_address - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.customer_demographics - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometNativeScan parquet spark_catalog.default.household_demographics - -Comet accelerated 46 out of 47 eligible operators (97%). Final plan contains 1 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_2/q91.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_2/q91.native_iceberg_compat/extended.txt deleted file mode 100644 index 961299b3d9..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_2/q91.native_iceberg_compat/extended.txt +++ /dev/null @@ -1,51 +0,0 @@ -CometNativeColumnarToRow -+- CometSort - +- CometExchange - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.call_center - : : : : : +- CometBroadcastExchange - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns - : : : : : +- 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.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.customer_demographics - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.household_demographics - -Comet accelerated 45 out of 47 eligible operators (95%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_2/q92.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_2/q92.native_datafusion/extended.txt deleted file mode 100644 index bcf7db282b..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_2/q92.native_datafusion/extended.txt +++ /dev/null @@ -1,41 +0,0 @@ -CometNativeColumnarToRow -+- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : +- CometSubqueryBroadcast - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.item - : +- CometBroadcastExchange - : +- CometFilter - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : +- ReusedSubquery - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.date_dim - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometNativeScan parquet spark_catalog.default.date_dim - -Comet accelerated 36 out of 38 eligible operators (94%). Final plan contains 1 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_2/q92.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_2/q92.native_iceberg_compat/extended.txt deleted file mode 100644 index 0fb6e6c158..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_2/q92.native_iceberg_compat/extended.txt +++ /dev/null @@ -1,42 +0,0 @@ -CometNativeColumnarToRow -+- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- 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 - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : +- CometBroadcastExchange - : +- CometFilter - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- 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.date_dim - -Comet accelerated 35 out of 38 eligible operators (92%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_2/q93.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_2/q93.native_datafusion/extended.txt deleted file mode 100644 index 1496f26233..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_2/q93.native_datafusion/extended.txt +++ /dev/null @@ -1,24 +0,0 @@ -CometNativeColumnarToRow -+- CometTakeOrderedAndProject - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometSortMergeJoin - : :- CometSort - : : +- CometExchange - : : +- CometProject - : : +- 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.reason - -Comet accelerated 21 out of 21 eligible operators (100%). Final plan contains 1 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_2/q93.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_2/q93.native_iceberg_compat/extended.txt deleted file mode 100644 index 79acf0e4eb..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_2/q93.native_iceberg_compat/extended.txt +++ /dev/null @@ -1,24 +0,0 @@ -CometNativeColumnarToRow -+- CometTakeOrderedAndProject - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometSortMergeJoin - : :- CometSort - : : +- CometExchange - : : +- CometProject - : : +- 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.reason - -Comet accelerated 21 out of 21 eligible operators (100%). Final plan contains 1 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_2/q94.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_2/q94.native_datafusion/extended.txt deleted file mode 100644 index 11de43f62c..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_2/q94.native_datafusion/extended.txt +++ /dev/null @@ -1,42 +0,0 @@ -CometNativeColumnarToRow -+- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometHashAggregate - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometSortMergeJoin - : : : :- CometProject - : : : : +- CometSortMergeJoin - : : : : :- CometSort - : : : : : +- CometExchange - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : : +- CometSort - : : : : +- CometExchange - : : : : +- CometProject - : : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : +- CometSort - : : : +- CometExchange - : : : +- CometProject - : : : +- CometNativeScan parquet spark_catalog.default.web_returns - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.customer_address - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometNativeScan parquet spark_catalog.default.web_site - -Comet accelerated 39 out of 39 eligible operators (100%). Final plan contains 1 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_2/q94.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_2/q94.native_iceberg_compat/extended.txt deleted file mode 100644 index a89558eb80..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_2/q94.native_iceberg_compat/extended.txt +++ /dev/null @@ -1,42 +0,0 @@ -CometNativeColumnarToRow -+- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometHashAggregate - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometSortMergeJoin - : : : :- CometProject - : : : : +- CometSortMergeJoin - : : : : :- CometSort - : : : : : +- CometExchange - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : +- CometSort - : : : : +- CometExchange - : : : : +- CometProject - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : +- CometSort - : : : +- CometExchange - : : : +- CometProject - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns - : : +- 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_address - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_site - -Comet accelerated 39 out of 39 eligible operators (100%). Final plan contains 1 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_2/q95.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_2/q95.native_datafusion/extended.txt deleted file mode 100644 index 45ec12b82b..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_2/q95.native_datafusion/extended.txt +++ /dev/null @@ -1,64 +0,0 @@ -CometNativeColumnarToRow -+- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometHashAggregate - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometSortMergeJoin - : : : :- CometSortMergeJoin - : : : : :- CometSort - : : : : : +- CometExchange - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : : +- CometProject - : : : : +- CometSortMergeJoin - : : : : :- CometSort - : : : : : +- CometExchange - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : : +- CometSort - : : : : +- CometExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : +- CometProject - : : : +- CometSortMergeJoin - : : : :- CometSort - : : : : +- CometExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.web_returns - : : : +- CometProject - : : : +- CometSortMergeJoin - : : : :- CometSort - : : : : +- CometExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : +- CometSort - : : : +- CometExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.customer_address - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometNativeScan parquet spark_catalog.default.web_site - -Comet accelerated 61 out of 61 eligible operators (100%). Final plan contains 1 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_2/q95.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_2/q95.native_iceberg_compat/extended.txt deleted file mode 100644 index 9a692023ae..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_2/q95.native_iceberg_compat/extended.txt +++ /dev/null @@ -1,64 +0,0 @@ -CometNativeColumnarToRow -+- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometHashAggregate - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometSortMergeJoin - : : : :- CometSortMergeJoin - : : : : :- CometSort - : : : : : +- CometExchange - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : +- CometProject - : : : : +- CometSortMergeJoin - : : : : :- CometSort - : : : : : +- CometExchange - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : +- CometSort - : : : : +- CometExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : +- CometProject - : : : +- CometSortMergeJoin - : : : :- CometSort - : : : : +- CometExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns - : : : +- CometProject - : : : +- CometSortMergeJoin - : : : :- CometSort - : : : : +- CometExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : +- CometSort - : : : +- CometExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : +- 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_address - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_site - -Comet accelerated 61 out of 61 eligible operators (100%). Final plan contains 1 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_2/q96.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_2/q96.native_datafusion/extended.txt deleted file mode 100644 index 9ee165830a..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_2/q96.native_datafusion/extended.txt +++ /dev/null @@ -1,27 +0,0 @@ -CometNativeColumnarToRow -+- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.household_demographics - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.time_dim - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometNativeScan parquet spark_catalog.default.store - -Comet accelerated 24 out of 24 eligible operators (100%). Final plan contains 1 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_2/q96.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_2/q96.native_iceberg_compat/extended.txt deleted file mode 100644 index 0f07a6473f..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_2/q96.native_iceberg_compat/extended.txt +++ /dev/null @@ -1,27 +0,0 @@ -CometNativeColumnarToRow -+- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.household_demographics - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.time_dim - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - -Comet accelerated 24 out of 24 eligible operators (100%). Final plan contains 1 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_2/q97.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_2/q97.native_datafusion/extended.txt deleted file mode 100644 index a56bd059fa..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_2/q97.native_datafusion/extended.txt +++ /dev/null @@ -1,36 +0,0 @@ -CometNativeColumnarToRow -+- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometSortMergeJoin - :- CometSort - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometNativeScan parquet spark_catalog.default.store_sales - : : +- CometSubqueryBroadcast - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.date_dim - +- CometSort - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometNativeScan parquet spark_catalog.default.catalog_sales - : +- ReusedSubquery - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometNativeScan parquet spark_catalog.default.date_dim - -Comet accelerated 31 out of 33 eligible operators (93%). Final plan contains 1 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_2/q97.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_2/q97.native_iceberg_compat/extended.txt deleted file mode 100644 index 4aa20a3755..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_2/q97.native_iceberg_compat/extended.txt +++ /dev/null @@ -1,37 +0,0 @@ -CometNativeColumnarToRow -+- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometSortMergeJoin - :- CometSort - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- 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 - +- CometSort - +- 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 - -Comet accelerated 30 out of 33 eligible operators (90%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_2/q98.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_2/q98.native_datafusion/extended.txt deleted file mode 100644 index f1fc5ff9a3..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_2/q98.native_datafusion/extended.txt +++ /dev/null @@ -1,33 +0,0 @@ -CometNativeColumnarToRow -+- CometProject - +- CometSort - +- CometColumnarExchange - +- Project - +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] - +- CometNativeColumnarToRow - +- CometSort - +- CometExchange - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : +- CometSubqueryBroadcast - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.item - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometNativeScan parquet spark_catalog.default.date_dim - -Comet accelerated 26 out of 29 eligible operators (89%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_2/q98.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_2/q98.native_iceberg_compat/extended.txt deleted file mode 100644 index a18137ef10..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_2/q98.native_iceberg_compat/extended.txt +++ /dev/null @@ -1,34 +0,0 @@ -CometNativeColumnarToRow -+- CometProject - +- CometSort - +- CometColumnarExchange - +- Project - +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] - +- CometNativeColumnarToRow - +- CometSort - +- CometExchange - +- 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.item - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - -Comet accelerated 25 out of 29 eligible operators (86%). Final plan contains 3 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_2/q99.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_2/q99.native_datafusion/extended.txt deleted file mode 100644 index c236a1647f..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_2/q99.native_datafusion/extended.txt +++ /dev/null @@ -1,31 +0,0 @@ -CometNativeColumnarToRow -+- CometTakeOrderedAndProject - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : +- CometBroadcastExchange - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.warehouse - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.ship_mode - : +- CometBroadcastExchange - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.call_center - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometNativeScan parquet spark_catalog.default.date_dim - -Comet accelerated 28 out of 28 eligible operators (100%). Final plan contains 1 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_2/q99.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_2/q99.native_iceberg_compat/extended.txt deleted file mode 100644 index 47e040af97..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_2/q99.native_iceberg_compat/extended.txt +++ /dev/null @@ -1,31 +0,0 @@ -CometNativeColumnarToRow -+- CometTakeOrderedAndProject - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : +- CometBroadcastExchange - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.warehouse - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.ship_mode - : +- CometBroadcastExchange - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.call_center - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - -Comet accelerated 28 out of 28 eligible operators (100%). Final plan contains 1 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_2/q10a.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_2/q10a.native_datafusion/extended.txt deleted file mode 100644 index f27de2fdbb..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_2/q10a.native_datafusion/extended.txt +++ /dev/null @@ -1,55 +0,0 @@ -CometNativeColumnarToRow -+- CometTakeOrderedAndProject - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.customer - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometNativeScan parquet spark_catalog.default.store_sales - : : : : +- CometSubqueryBroadcast - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometUnion - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometNativeScan parquet spark_catalog.default.web_sales - : : : : +- ReusedSubquery - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : +- ReusedSubquery - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.customer_address - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometNativeScan parquet spark_catalog.default.customer_demographics - -Comet accelerated 49 out of 52 eligible operators (94%). Final plan contains 1 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_2/q10a.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_2/q10a.native_iceberg_compat/extended.txt deleted file mode 100644 index bdbfb0d6dc..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_2/q10a.native_iceberg_compat/extended.txt +++ /dev/null @@ -1,56 +0,0 @@ -CometNativeColumnarToRow -+- CometTakeOrderedAndProject - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- 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 - : : +- CometBroadcastExchange - : : +- CometUnion - : : :- 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 - : : +- 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 - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics - -Comet accelerated 48 out of 52 eligible operators (92%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_2/q11.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_2/q11.native_datafusion/extended.txt deleted file mode 100644 index cfe634568f..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_2/q11.native_datafusion/extended.txt +++ /dev/null @@ -1,88 +0,0 @@ -CometNativeColumnarToRow -+- CometTakeOrderedAndProject - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometHashAggregate - : : : +- CometExchange - : : : +- CometHashAggregate - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.customer - : : : : +- CometBroadcastExchange - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : : +- CometSubqueryBroadcast - : : : : +- CometBroadcastExchange - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometHashAggregate - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.customer - : : : +- CometBroadcastExchange - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : +- CometSubqueryBroadcast - : : : +- CometBroadcastExchange - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometFilter - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.customer - : : +- CometBroadcastExchange - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : +- ReusedSubquery - : +- CometBroadcastExchange - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.date_dim - +- CometBroadcastExchange - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.customer - : +- CometBroadcastExchange - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.web_sales - : +- ReusedSubquery - +- CometBroadcastExchange - +- CometFilter - +- CometNativeScan parquet spark_catalog.default.date_dim - -Comet accelerated 81 out of 85 eligible operators (95%). Final plan contains 1 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_2/q11.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_2/q11.native_iceberg_compat/extended.txt deleted file mode 100644 index 922864baa1..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_2/q11.native_iceberg_compat/extended.txt +++ /dev/null @@ -1,90 +0,0 @@ -CometNativeColumnarToRow -+- CometTakeOrderedAndProject - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometHashAggregate - : : : +- CometExchange - : : : +- CometHashAggregate - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer - : : : : +- CometBroadcastExchange - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometHashAggregate - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer - : : : +- CometBroadcastExchange - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometFilter - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer - : : +- CometBroadcastExchange - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : +- ReusedSubquery - : +- CometBroadcastExchange - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - +- CometBroadcastExchange - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer - : +- CometBroadcastExchange - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : +- ReusedSubquery - +- CometBroadcastExchange - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - -Comet accelerated 79 out of 85 eligible operators (92%). Final plan contains 3 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_2/q12.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_2/q12.native_datafusion/extended.txt deleted file mode 100644 index a6cc526735..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_2/q12.native_datafusion/extended.txt +++ /dev/null @@ -1,30 +0,0 @@ -TakeOrderedAndProject -+- 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 - :- CometProject - : +- CometBroadcastHashJoin - : :- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : +- CometSubqueryBroadcast - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.item - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometNativeScan parquet spark_catalog.default.date_dim - -Comet accelerated 23 out of 27 eligible operators (85%). Final plan contains 1 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_2/q12.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_2/q12.native_iceberg_compat/extended.txt deleted file mode 100644 index eb85a80eb1..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_2/q12.native_iceberg_compat/extended.txt +++ /dev/null @@ -1,31 +0,0 @@ -TakeOrderedAndProject -+- 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 - :- 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.item - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - -Comet accelerated 22 out of 27 eligible operators (81%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_2/q14.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_2/q14.native_datafusion/extended.txt deleted file mode 100644 index 85cda8b553..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_2/q14.native_datafusion/extended.txt +++ /dev/null @@ -1,345 +0,0 @@ -CometNativeColumnarToRow -+- CometTakeOrderedAndProject - +- CometBroadcastHashJoin - :- CometFilter - : : +- Subquery - : : +- CometNativeColumnarToRow - : : +- CometHashAggregate - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometUnion - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometNativeScan parquet spark_catalog.default.store_sales - : : : : +- ReusedSubquery - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : +- ReusedSubquery - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometNativeScan parquet spark_catalog.default.web_sales - : : : +- ReusedSubquery - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : : +- CometSubqueryBroadcast - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : : +- ReusedSubquery - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : +- Subquery - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : +- CometBroadcastExchange - : : : +- CometBroadcastHashJoin - : : : :- CometHashAggregate - : : : : +- CometExchange - : : : : +- CometHashAggregate - : : : : +- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : : : : +- CometSubqueryBroadcast - : : : : : : +- CometBroadcastExchange - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : : +- CometBroadcastExchange - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometProject - : : : : : : +- CometBroadcastHashJoin - : : : : : : :- CometFilter - : : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : : : : +- ReusedSubquery - : : : : : : +- CometBroadcastExchange - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : : : +- ReusedSubquery - : : : : +- CometBroadcastExchange - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.item - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.item - : : +- CometBroadcastExchange - : : +- CometBroadcastHashJoin - : : :- CometHashAggregate - : : : +- CometExchange - : : : +- CometHashAggregate - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : : : +- CometSubqueryBroadcast - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : : : +- ReusedSubquery - : : : : : +- CometBroadcastExchange - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : : +- ReusedSubquery - : : : +- CometBroadcastExchange - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.item - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : : +- ReusedSubquery - : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- Subquery - : +- CometNativeColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.date_dim - +- CometBroadcastExchange - +- CometFilter - : +- ReusedSubquery - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : +- CometSubqueryBroadcast - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : : +- ReusedSubquery - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- Subquery - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.item - : : +- CometBroadcastExchange - : : +- CometBroadcastHashJoin - : : :- CometHashAggregate - : : : +- CometExchange - : : : +- CometHashAggregate - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : : : +- CometSubqueryBroadcast - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : : : +- ReusedSubquery - : : : : : +- CometBroadcastExchange - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : : +- ReusedSubquery - : : : +- CometBroadcastExchange - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.item - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometBroadcastHashJoin - : :- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.item - : +- CometBroadcastExchange - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.item - : +- CometBroadcastExchange - : +- CometBroadcastHashJoin - : :- CometHashAggregate - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : : +- CometSubqueryBroadcast - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : : +- ReusedSubquery - : : : : +- CometBroadcastExchange - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : +- ReusedSubquery - : : +- CometBroadcastExchange - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.item - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.date_dim - +- CometBroadcastExchange - +- CometProject - +- CometFilter - : +- ReusedSubquery - +- CometNativeScan parquet spark_catalog.default.date_dim - +- Subquery - +- CometNativeColumnarToRow - +- CometProject - +- CometFilter - +- CometNativeScan parquet spark_catalog.default.date_dim - -Comet accelerated 310 out of 337 eligible operators (91%). Final plan contains 6 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_2/q14.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_2/q14.native_iceberg_compat/extended.txt deleted file mode 100644 index 7a4afd3d2b..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_2/q14.native_iceberg_compat/extended.txt +++ /dev/null @@ -1,343 +0,0 @@ -CometNativeColumnarToRow -+- CometTakeOrderedAndProject - +- CometBroadcastHashJoin - :- CometFilter - : : +- Subquery - : : +- CometNativeColumnarToRow - : : +- CometHashAggregate - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometUnion - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : +- ReusedSubquery - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : :- 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 - : : +- 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 - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : : +- ReusedSubquery - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- Subquery - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : +- CometBroadcastExchange - : : : +- 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 - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometProject - : : : : : : +- CometBroadcastHashJoin - : : : : : : :- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : : : +- ReusedSubquery - : : : : : : +- CometBroadcastExchange - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : : +- CometBroadcastExchange - : : : : : +- 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 - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : : +- ReusedSubquery - : : : : +- CometBroadcastExchange - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : +- CometBroadcastExchange - : : +- 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 - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : : +- ReusedSubquery - : : : : : +- CometBroadcastExchange - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : +- CometBroadcastExchange - : : : : +- 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 - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : +- ReusedSubquery - : : : +- CometBroadcastExchange - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : : +- ReusedSubquery - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- ReusedSubquery - +- CometBroadcastExchange - +- CometFilter - : +- ReusedSubquery - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : : +- ReusedSubquery - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- Subquery - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : +- CometBroadcastExchange - : : +- 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 - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : : +- ReusedSubquery - : : : : : +- CometBroadcastExchange - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : +- CometBroadcastExchange - : : : : +- 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 - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : +- ReusedSubquery - : : : +- CometBroadcastExchange - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometBroadcastHashJoin - : :- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : +- CometBroadcastExchange - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : +- CometBroadcastExchange - : +- 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 - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : +- ReusedSubquery - : : : : +- CometBroadcastExchange - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : +- CometBroadcastExchange - : : : +- 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 - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : +- ReusedSubquery - : : +- CometBroadcastExchange - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - +- CometBroadcastExchange - +- CometProject - +- CometFilter - : +- ReusedSubquery - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - +- ReusedSubquery - -Comet accelerated 298 out of 331 eligible operators (90%). Final plan contains 10 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_2/q14a.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_2/q14a.native_datafusion/extended.txt deleted file mode 100644 index dd43130d46..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_2/q14a.native_datafusion/extended.txt +++ /dev/null @@ -1,2173 +0,0 @@ -CometNativeColumnarToRow -+- CometTakeOrderedAndProject - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometUnion - :- CometHashAggregate - : +- CometHashAggregate - : +- CometUnion - : :- CometFilter - : : : +- Subquery - : : : +- CometNativeColumnarToRow - : : : +- CometHashAggregate - : : : +- CometExchange - : : : +- CometHashAggregate - : : : +- CometUnion - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometNativeScan parquet spark_catalog.default.store_sales - : : : : : +- ReusedSubquery - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : : +- CometSubqueryBroadcast - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometNativeScan parquet spark_catalog.default.web_sales - : : : : +- ReusedSubquery - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- CometHashAggregate - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : : : +- CometSubqueryBroadcast - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : +- CometBroadcastExchange - : : : : +- CometBroadcastHashJoin - : : : : :- CometHashAggregate - : : : : : +- CometExchange - : : : : : +- CometHashAggregate - : : : : : +- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometProject - : : : : : : +- CometBroadcastHashJoin - : : : : : : :- CometFilter - : : : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : : : : : +- CometSubqueryBroadcast - : : : : : : : +- CometBroadcastExchange - : : : : : : : +- CometProject - : : : : : : : +- CometFilter - : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : : : +- CometBroadcastExchange - : : : : : : +- CometBroadcastHashJoin - : : : : : : :- CometFilter - : : : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : : : +- CometBroadcastExchange - : : : : : : +- CometProject - : : : : : : +- CometBroadcastHashJoin - : : : : : : :- CometProject - : : : : : : : +- CometBroadcastHashJoin - : : : : : : : :- CometFilter - : : : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : : : : : +- ReusedSubquery - : : : : : : : +- CometBroadcastExchange - : : : : : : : +- CometFilter - : : : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : : : +- CometBroadcastExchange - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : : : : +- ReusedSubquery - : : : : : +- CometBroadcastExchange - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : +- CometBroadcastExchange - : : : +- CometBroadcastHashJoin - : : : :- CometHashAggregate - : : : : +- CometExchange - : : : : +- CometHashAggregate - : : : : +- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : : : : +- CometSubqueryBroadcast - : : : : : : +- CometBroadcastExchange - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : : +- CometBroadcastExchange - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometProject - : : : : : : +- CometBroadcastHashJoin - : : : : : : :- CometFilter - : : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : : : : +- ReusedSubquery - : : : : : : +- CometBroadcastExchange - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : : : +- ReusedSubquery - : : : : +- CometBroadcastExchange - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : :- CometFilter - : : : +- ReusedSubquery - : : +- CometHashAggregate - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : : +- ReusedSubquery - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : +- CometBroadcastExchange - : : : : +- CometBroadcastHashJoin - : : : : :- CometHashAggregate - : : : : : +- CometExchange - : : : : : +- CometHashAggregate - : : : : : +- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometProject - : : : : : : +- CometBroadcastHashJoin - : : : : : : :- CometFilter - : : : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : : : : : +- CometSubqueryBroadcast - : : : : : : : +- CometBroadcastExchange - : : : : : : : +- CometProject - : : : : : : : +- CometFilter - : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : : : +- CometBroadcastExchange - : : : : : : +- CometBroadcastHashJoin - : : : : : : :- CometFilter - : : : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : : : +- CometBroadcastExchange - : : : : : : +- CometProject - : : : : : : +- CometBroadcastHashJoin - : : : : : : :- CometProject - : : : : : : : +- CometBroadcastHashJoin - : : : : : : : :- CometFilter - : : : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : : : : : +- ReusedSubquery - : : : : : : : +- CometBroadcastExchange - : : : : : : : +- CometFilter - : : : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : : : +- CometBroadcastExchange - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : : : : +- ReusedSubquery - : : : : : +- CometBroadcastExchange - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : +- CometBroadcastExchange - : : : +- CometBroadcastHashJoin - : : : :- CometHashAggregate - : : : : +- CometExchange - : : : : +- CometHashAggregate - : : : : +- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : : : : +- CometSubqueryBroadcast - : : : : : : +- CometBroadcastExchange - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : : +- CometBroadcastExchange - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometProject - : : : : : : +- CometBroadcastHashJoin - : : : : : : :- CometFilter - : : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : : : : +- ReusedSubquery - : : : : : : +- CometBroadcastExchange - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : : : +- ReusedSubquery - : : : : +- CometBroadcastExchange - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- CometFilter - : : +- ReusedSubquery - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : : +- ReusedSubquery - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : +- CometBroadcastExchange - : : : +- CometBroadcastHashJoin - : : : :- CometHashAggregate - : : : : +- CometExchange - : : : : +- CometHashAggregate - : : : : +- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : : : : +- CometSubqueryBroadcast - : : : : : : +- CometBroadcastExchange - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : : +- CometBroadcastExchange - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometProject - : : : : : : +- CometBroadcastHashJoin - : : : : : : :- CometFilter - : : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : : : : +- ReusedSubquery - : : : : : : +- CometBroadcastExchange - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : : : +- ReusedSubquery - : : : : +- CometBroadcastExchange - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.item - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.item - : : +- CometBroadcastExchange - : : +- CometBroadcastHashJoin - : : :- CometHashAggregate - : : : +- CometExchange - : : : +- CometHashAggregate - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : : : +- CometSubqueryBroadcast - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : : : +- ReusedSubquery - : : : : : +- CometBroadcastExchange - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : : +- ReusedSubquery - : : : +- CometBroadcastExchange - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.item - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.date_dim - :- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometHashAggregate - : +- CometHashAggregate - : +- CometUnion - : :- CometFilter - : : : +- ReusedSubquery - : : +- CometHashAggregate - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : : : +- CometSubqueryBroadcast - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : +- CometBroadcastExchange - : : : : +- CometBroadcastHashJoin - : : : : :- CometHashAggregate - : : : : : +- CometExchange - : : : : : +- CometHashAggregate - : : : : : +- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometProject - : : : : : : +- CometBroadcastHashJoin - : : : : : : :- CometFilter - : : : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : : : : : +- CometSubqueryBroadcast - : : : : : : : +- CometBroadcastExchange - : : : : : : : +- CometProject - : : : : : : : +- CometFilter - : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : : : +- CometBroadcastExchange - : : : : : : +- CometBroadcastHashJoin - : : : : : : :- CometFilter - : : : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : : : +- CometBroadcastExchange - : : : : : : +- CometProject - : : : : : : +- CometBroadcastHashJoin - : : : : : : :- CometProject - : : : : : : : +- CometBroadcastHashJoin - : : : : : : : :- CometFilter - : : : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : : : : : +- ReusedSubquery - : : : : : : : +- CometBroadcastExchange - : : : : : : : +- CometFilter - : : : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : : : +- CometBroadcastExchange - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : : : : +- ReusedSubquery - : : : : : +- CometBroadcastExchange - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : +- CometBroadcastExchange - : : : +- CometBroadcastHashJoin - : : : :- CometHashAggregate - : : : : +- CometExchange - : : : : +- CometHashAggregate - : : : : +- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : : : : +- CometSubqueryBroadcast - : : : : : : +- CometBroadcastExchange - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : : +- CometBroadcastExchange - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometProject - : : : : : : +- CometBroadcastHashJoin - : : : : : : :- CometFilter - : : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : : : : +- ReusedSubquery - : : : : : : +- CometBroadcastExchange - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : : : +- ReusedSubquery - : : : : +- CometBroadcastExchange - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : :- CometFilter - : : : +- ReusedSubquery - : : +- CometHashAggregate - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : : +- ReusedSubquery - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : +- CometBroadcastExchange - : : : : +- CometBroadcastHashJoin - : : : : :- CometHashAggregate - : : : : : +- CometExchange - : : : : : +- CometHashAggregate - : : : : : +- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometProject - : : : : : : +- CometBroadcastHashJoin - : : : : : : :- CometFilter - : : : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : : : : : +- CometSubqueryBroadcast - : : : : : : : +- CometBroadcastExchange - : : : : : : : +- CometProject - : : : : : : : +- CometFilter - : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : : : +- CometBroadcastExchange - : : : : : : +- CometBroadcastHashJoin - : : : : : : :- CometFilter - : : : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : : : +- CometBroadcastExchange - : : : : : : +- CometProject - : : : : : : +- CometBroadcastHashJoin - : : : : : : :- CometProject - : : : : : : : +- CometBroadcastHashJoin - : : : : : : : :- CometFilter - : : : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : : : : : +- ReusedSubquery - : : : : : : : +- CometBroadcastExchange - : : : : : : : +- CometFilter - : : : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : : : +- CometBroadcastExchange - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : : : : +- ReusedSubquery - : : : : : +- CometBroadcastExchange - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : +- CometBroadcastExchange - : : : +- CometBroadcastHashJoin - : : : :- CometHashAggregate - : : : : +- CometExchange - : : : : +- CometHashAggregate - : : : : +- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : : : : +- CometSubqueryBroadcast - : : : : : : +- CometBroadcastExchange - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : : +- CometBroadcastExchange - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometProject - : : : : : : +- CometBroadcastHashJoin - : : : : : : :- CometFilter - : : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : : : : +- ReusedSubquery - : : : : : : +- CometBroadcastExchange - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : : : +- ReusedSubquery - : : : : +- CometBroadcastExchange - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- CometFilter - : : +- ReusedSubquery - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : : +- ReusedSubquery - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : +- CometBroadcastExchange - : : : +- CometBroadcastHashJoin - : : : :- CometHashAggregate - : : : : +- CometExchange - : : : : +- CometHashAggregate - : : : : +- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : : : : +- CometSubqueryBroadcast - : : : : : : +- CometBroadcastExchange - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : : +- CometBroadcastExchange - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometProject - : : : : : : +- CometBroadcastHashJoin - : : : : : : :- CometFilter - : : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : : : : +- ReusedSubquery - : : : : : : +- CometBroadcastExchange - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : : : +- ReusedSubquery - : : : : +- CometBroadcastExchange - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.item - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.item - : : +- CometBroadcastExchange - : : +- CometBroadcastHashJoin - : : :- CometHashAggregate - : : : +- CometExchange - : : : +- CometHashAggregate - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : : : +- CometSubqueryBroadcast - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : : : +- ReusedSubquery - : : : : : +- CometBroadcastExchange - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : : +- ReusedSubquery - : : : +- CometBroadcastExchange - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.item - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.date_dim - :- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometHashAggregate - : +- CometHashAggregate - : +- CometUnion - : :- CometFilter - : : : +- ReusedSubquery - : : +- CometHashAggregate - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : : : +- CometSubqueryBroadcast - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : +- CometBroadcastExchange - : : : : +- CometBroadcastHashJoin - : : : : :- CometHashAggregate - : : : : : +- CometExchange - : : : : : +- CometHashAggregate - : : : : : +- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometProject - : : : : : : +- CometBroadcastHashJoin - : : : : : : :- CometFilter - : : : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : : : : : +- CometSubqueryBroadcast - : : : : : : : +- CometBroadcastExchange - : : : : : : : +- CometProject - : : : : : : : +- CometFilter - : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : : : +- CometBroadcastExchange - : : : : : : +- CometBroadcastHashJoin - : : : : : : :- CometFilter - : : : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : : : +- CometBroadcastExchange - : : : : : : +- CometProject - : : : : : : +- CometBroadcastHashJoin - : : : : : : :- CometProject - : : : : : : : +- CometBroadcastHashJoin - : : : : : : : :- CometFilter - : : : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : : : : : +- ReusedSubquery - : : : : : : : +- CometBroadcastExchange - : : : : : : : +- CometFilter - : : : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : : : +- CometBroadcastExchange - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : : : : +- ReusedSubquery - : : : : : +- CometBroadcastExchange - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : +- CometBroadcastExchange - : : : +- CometBroadcastHashJoin - : : : :- CometHashAggregate - : : : : +- CometExchange - : : : : +- CometHashAggregate - : : : : +- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : : : : +- CometSubqueryBroadcast - : : : : : : +- CometBroadcastExchange - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : : +- CometBroadcastExchange - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometProject - : : : : : : +- CometBroadcastHashJoin - : : : : : : :- CometFilter - : : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : : : : +- ReusedSubquery - : : : : : : +- CometBroadcastExchange - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : : : +- ReusedSubquery - : : : : +- CometBroadcastExchange - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : :- CometFilter - : : : +- ReusedSubquery - : : +- CometHashAggregate - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : : +- ReusedSubquery - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : +- CometBroadcastExchange - : : : : +- CometBroadcastHashJoin - : : : : :- CometHashAggregate - : : : : : +- CometExchange - : : : : : +- CometHashAggregate - : : : : : +- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometProject - : : : : : : +- CometBroadcastHashJoin - : : : : : : :- CometFilter - : : : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : : : : : +- CometSubqueryBroadcast - : : : : : : : +- CometBroadcastExchange - : : : : : : : +- CometProject - : : : : : : : +- CometFilter - : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : : : +- CometBroadcastExchange - : : : : : : +- CometBroadcastHashJoin - : : : : : : :- CometFilter - : : : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : : : +- CometBroadcastExchange - : : : : : : +- CometProject - : : : : : : +- CometBroadcastHashJoin - : : : : : : :- CometProject - : : : : : : : +- CometBroadcastHashJoin - : : : : : : : :- CometFilter - : : : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : : : : : +- ReusedSubquery - : : : : : : : +- CometBroadcastExchange - : : : : : : : +- CometFilter - : : : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : : : +- CometBroadcastExchange - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : : : : +- ReusedSubquery - : : : : : +- CometBroadcastExchange - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : +- CometBroadcastExchange - : : : +- CometBroadcastHashJoin - : : : :- CometHashAggregate - : : : : +- CometExchange - : : : : +- CometHashAggregate - : : : : +- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : : : : +- CometSubqueryBroadcast - : : : : : : +- CometBroadcastExchange - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : : +- CometBroadcastExchange - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometProject - : : : : : : +- CometBroadcastHashJoin - : : : : : : :- CometFilter - : : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : : : : +- ReusedSubquery - : : : : : : +- CometBroadcastExchange - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : : : +- ReusedSubquery - : : : : +- CometBroadcastExchange - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- CometFilter - : : +- ReusedSubquery - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : : +- ReusedSubquery - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : +- CometBroadcastExchange - : : : +- CometBroadcastHashJoin - : : : :- CometHashAggregate - : : : : +- CometExchange - : : : : +- CometHashAggregate - : : : : +- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : : : : +- CometSubqueryBroadcast - : : : : : : +- CometBroadcastExchange - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : : +- CometBroadcastExchange - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometProject - : : : : : : +- CometBroadcastHashJoin - : : : : : : :- CometFilter - : : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : : : : +- ReusedSubquery - : : : : : : +- CometBroadcastExchange - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : : : +- ReusedSubquery - : : : : +- CometBroadcastExchange - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.item - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.item - : : +- CometBroadcastExchange - : : +- CometBroadcastHashJoin - : : :- CometHashAggregate - : : : +- CometExchange - : : : +- CometHashAggregate - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : : : +- CometSubqueryBroadcast - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : : : +- ReusedSubquery - : : : : : +- CometBroadcastExchange - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : : +- ReusedSubquery - : : : +- CometBroadcastExchange - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.item - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.date_dim - :- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometHashAggregate - : +- CometHashAggregate - : +- CometUnion - : :- CometFilter - : : : +- ReusedSubquery - : : +- CometHashAggregate - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : : : +- CometSubqueryBroadcast - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : +- CometBroadcastExchange - : : : : +- CometBroadcastHashJoin - : : : : :- CometHashAggregate - : : : : : +- CometExchange - : : : : : +- CometHashAggregate - : : : : : +- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometProject - : : : : : : +- CometBroadcastHashJoin - : : : : : : :- CometFilter - : : : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : : : : : +- CometSubqueryBroadcast - : : : : : : : +- CometBroadcastExchange - : : : : : : : +- CometProject - : : : : : : : +- CometFilter - : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : : : +- CometBroadcastExchange - : : : : : : +- CometBroadcastHashJoin - : : : : : : :- CometFilter - : : : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : : : +- CometBroadcastExchange - : : : : : : +- CometProject - : : : : : : +- CometBroadcastHashJoin - : : : : : : :- CometProject - : : : : : : : +- CometBroadcastHashJoin - : : : : : : : :- CometFilter - : : : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : : : : : +- ReusedSubquery - : : : : : : : +- CometBroadcastExchange - : : : : : : : +- CometFilter - : : : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : : : +- CometBroadcastExchange - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : : : : +- ReusedSubquery - : : : : : +- CometBroadcastExchange - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : +- CometBroadcastExchange - : : : +- CometBroadcastHashJoin - : : : :- CometHashAggregate - : : : : +- CometExchange - : : : : +- CometHashAggregate - : : : : +- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : : : : +- CometSubqueryBroadcast - : : : : : : +- CometBroadcastExchange - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : : +- CometBroadcastExchange - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometProject - : : : : : : +- CometBroadcastHashJoin - : : : : : : :- CometFilter - : : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : : : : +- ReusedSubquery - : : : : : : +- CometBroadcastExchange - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : : : +- ReusedSubquery - : : : : +- CometBroadcastExchange - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : :- CometFilter - : : : +- ReusedSubquery - : : +- CometHashAggregate - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : : +- ReusedSubquery - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : +- CometBroadcastExchange - : : : : +- CometBroadcastHashJoin - : : : : :- CometHashAggregate - : : : : : +- CometExchange - : : : : : +- CometHashAggregate - : : : : : +- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometProject - : : : : : : +- CometBroadcastHashJoin - : : : : : : :- CometFilter - : : : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : : : : : +- CometSubqueryBroadcast - : : : : : : : +- CometBroadcastExchange - : : : : : : : +- CometProject - : : : : : : : +- CometFilter - : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : : : +- CometBroadcastExchange - : : : : : : +- CometBroadcastHashJoin - : : : : : : :- CometFilter - : : : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : : : +- CometBroadcastExchange - : : : : : : +- CometProject - : : : : : : +- CometBroadcastHashJoin - : : : : : : :- CometProject - : : : : : : : +- CometBroadcastHashJoin - : : : : : : : :- CometFilter - : : : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : : : : : +- ReusedSubquery - : : : : : : : +- CometBroadcastExchange - : : : : : : : +- CometFilter - : : : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : : : +- CometBroadcastExchange - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : : : : +- ReusedSubquery - : : : : : +- CometBroadcastExchange - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : +- CometBroadcastExchange - : : : +- CometBroadcastHashJoin - : : : :- CometHashAggregate - : : : : +- CometExchange - : : : : +- CometHashAggregate - : : : : +- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : : : : +- CometSubqueryBroadcast - : : : : : : +- CometBroadcastExchange - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : : +- CometBroadcastExchange - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometProject - : : : : : : +- CometBroadcastHashJoin - : : : : : : :- CometFilter - : : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : : : : +- ReusedSubquery - : : : : : : +- CometBroadcastExchange - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : : : +- ReusedSubquery - : : : : +- CometBroadcastExchange - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- CometFilter - : : +- ReusedSubquery - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : : +- ReusedSubquery - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : +- CometBroadcastExchange - : : : +- CometBroadcastHashJoin - : : : :- CometHashAggregate - : : : : +- CometExchange - : : : : +- CometHashAggregate - : : : : +- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : : : : +- CometSubqueryBroadcast - : : : : : : +- CometBroadcastExchange - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : : +- CometBroadcastExchange - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometProject - : : : : : : +- CometBroadcastHashJoin - : : : : : : :- CometFilter - : : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : : : : +- ReusedSubquery - : : : : : : +- CometBroadcastExchange - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : : : +- ReusedSubquery - : : : : +- CometBroadcastExchange - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.item - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.item - : : +- CometBroadcastExchange - : : +- CometBroadcastHashJoin - : : :- CometHashAggregate - : : : +- CometExchange - : : : +- CometHashAggregate - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : : : +- CometSubqueryBroadcast - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : : : +- ReusedSubquery - : : : : : +- CometBroadcastExchange - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : : +- ReusedSubquery - : : : +- CometBroadcastExchange - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.item - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.date_dim - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometHashAggregate - +- CometHashAggregate - +- CometUnion - :- CometFilter - : : +- ReusedSubquery - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : : +- CometSubqueryBroadcast - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : +- CometBroadcastExchange - : : : +- CometBroadcastHashJoin - : : : :- CometHashAggregate - : : : : +- CometExchange - : : : : +- CometHashAggregate - : : : : +- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : : : : +- CometSubqueryBroadcast - : : : : : : +- CometBroadcastExchange - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : : +- CometBroadcastExchange - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometProject - : : : : : : +- CometBroadcastHashJoin - : : : : : : :- CometFilter - : : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : : : : +- ReusedSubquery - : : : : : : +- CometBroadcastExchange - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : : : +- ReusedSubquery - : : : : +- CometBroadcastExchange - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.item - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.item - : : +- CometBroadcastExchange - : : +- CometBroadcastHashJoin - : : :- CometHashAggregate - : : : +- CometExchange - : : : +- CometHashAggregate - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : : : +- CometSubqueryBroadcast - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : : : +- ReusedSubquery - : : : : : +- CometBroadcastExchange - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : : +- ReusedSubquery - : : : +- CometBroadcastExchange - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.item - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.date_dim - :- CometFilter - : : +- ReusedSubquery - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : +- ReusedSubquery - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : +- CometBroadcastExchange - : : : +- CometBroadcastHashJoin - : : : :- CometHashAggregate - : : : : +- CometExchange - : : : : +- CometHashAggregate - : : : : +- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : : : : +- CometSubqueryBroadcast - : : : : : : +- CometBroadcastExchange - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : : +- CometBroadcastExchange - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometProject - : : : : : : +- CometBroadcastHashJoin - : : : : : : :- CometFilter - : : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : : : : +- ReusedSubquery - : : : : : : +- CometBroadcastExchange - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : : : +- ReusedSubquery - : : : : +- CometBroadcastExchange - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.item - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.item - : : +- CometBroadcastExchange - : : +- CometBroadcastHashJoin - : : :- CometHashAggregate - : : : +- CometExchange - : : : +- CometHashAggregate - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : : : +- CometSubqueryBroadcast - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : : : +- ReusedSubquery - : : : : : +- CometBroadcastExchange - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : : +- ReusedSubquery - : : : +- CometBroadcastExchange - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.item - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.date_dim - +- CometFilter - : +- ReusedSubquery - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : +- ReusedSubquery - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.item - : : +- CometBroadcastExchange - : : +- CometBroadcastHashJoin - : : :- CometHashAggregate - : : : +- CometExchange - : : : +- CometHashAggregate - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : : : +- CometSubqueryBroadcast - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : : : +- ReusedSubquery - : : : : : +- CometBroadcastExchange - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : : +- ReusedSubquery - : : : +- CometBroadcastExchange - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.item - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometBroadcastHashJoin - : :- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.item - : +- CometBroadcastExchange - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.item - : +- CometBroadcastExchange - : +- CometBroadcastHashJoin - : :- CometHashAggregate - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : : +- CometSubqueryBroadcast - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : : +- ReusedSubquery - : : : : +- CometBroadcastExchange - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : +- ReusedSubquery - : : +- CometBroadcastExchange - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.item - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.date_dim - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometNativeScan parquet spark_catalog.default.date_dim - -Comet accelerated 2046 out of 2169 eligible operators (94%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_2/q14a.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_2/q14a.native_iceberg_compat/extended.txt deleted file mode 100644 index eaccfdfc70..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_2/q14a.native_iceberg_compat/extended.txt +++ /dev/null @@ -1,2209 +0,0 @@ -CometNativeColumnarToRow -+- CometTakeOrderedAndProject - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometUnion - :- CometHashAggregate - : +- CometHashAggregate - : +- CometUnion - : :- CometFilter - : : : +- Subquery - : : : +- CometNativeColumnarToRow - : : : +- CometHashAggregate - : : : +- CometExchange - : : : +- CometHashAggregate - : : : +- CometUnion - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : +- ReusedSubquery - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_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 - : : : +- 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 - : : +- CometHashAggregate - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- 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 - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : +- CometBroadcastExchange - : : : : +- 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 - : : : : : : +- CometBroadcastHashJoin - : : : : : : :- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : : : +- CometBroadcastExchange - : : : : : : +- CometProject - : : : : : : +- CometBroadcastHashJoin - : : : : : : :- CometProject - : : : : : : : +- CometBroadcastHashJoin - : : : : : : : :- CometFilter - : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : : : : +- ReusedSubquery - : : : : : : : +- CometBroadcastExchange - : : : : : : : +- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : : : +- CometBroadcastExchange - : : : : : : +- 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 - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : : : +- ReusedSubquery - : : : : : +- CometBroadcastExchange - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : +- CometBroadcastExchange - : : : +- 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 - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometProject - : : : : : : +- CometBroadcastHashJoin - : : : : : : :- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : : : +- ReusedSubquery - : : : : : : +- CometBroadcastExchange - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : : +- CometBroadcastExchange - : : : : : +- 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 - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : : +- ReusedSubquery - : : : : +- CometBroadcastExchange - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : +- CometBroadcastExchange - : : : +- 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 - : :- CometFilter - : : : +- ReusedSubquery - : : +- CometHashAggregate - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : +- ReusedSubquery - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : +- CometBroadcastExchange - : : : : +- 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 - : : : : : : +- CometBroadcastHashJoin - : : : : : : :- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : : : +- CometBroadcastExchange - : : : : : : +- CometProject - : : : : : : +- CometBroadcastHashJoin - : : : : : : :- CometProject - : : : : : : : +- CometBroadcastHashJoin - : : : : : : : :- CometFilter - : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : : : : +- ReusedSubquery - : : : : : : : +- CometBroadcastExchange - : : : : : : : +- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : : : +- CometBroadcastExchange - : : : : : : +- 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 - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : : : +- ReusedSubquery - : : : : : +- CometBroadcastExchange - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : +- CometBroadcastExchange - : : : +- 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 - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometProject - : : : : : : +- CometBroadcastHashJoin - : : : : : : :- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : : : +- ReusedSubquery - : : : : : : +- CometBroadcastExchange - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : : +- CometBroadcastExchange - : : : : : +- 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 - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : : +- ReusedSubquery - : : : : +- CometBroadcastExchange - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : +- CometBroadcastExchange - : : : +- 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 - : +- CometFilter - : : +- ReusedSubquery - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : +- ReusedSubquery - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : +- CometBroadcastExchange - : : : +- 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 - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometProject - : : : : : : +- CometBroadcastHashJoin - : : : : : : :- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : : : +- ReusedSubquery - : : : : : : +- CometBroadcastExchange - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : : +- CometBroadcastExchange - : : : : : +- 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 - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : : +- ReusedSubquery - : : : : +- CometBroadcastExchange - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : +- CometBroadcastExchange - : : +- 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 - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : : +- ReusedSubquery - : : : : : +- CometBroadcastExchange - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : +- CometBroadcastExchange - : : : : +- 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 - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : +- ReusedSubquery - : : : +- CometBroadcastExchange - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : +- CometBroadcastExchange - : : +- 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 - :- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometHashAggregate - : +- CometHashAggregate - : +- CometUnion - : :- CometFilter - : : : +- ReusedSubquery - : : +- CometHashAggregate - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- 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 - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : +- CometBroadcastExchange - : : : : +- 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 - : : : : : : +- CometBroadcastHashJoin - : : : : : : :- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : : : +- CometBroadcastExchange - : : : : : : +- CometProject - : : : : : : +- CometBroadcastHashJoin - : : : : : : :- CometProject - : : : : : : : +- CometBroadcastHashJoin - : : : : : : : :- CometFilter - : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : : : : +- ReusedSubquery - : : : : : : : +- CometBroadcastExchange - : : : : : : : +- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : : : +- CometBroadcastExchange - : : : : : : +- 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 - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : : : +- ReusedSubquery - : : : : : +- CometBroadcastExchange - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : +- CometBroadcastExchange - : : : +- 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 - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometProject - : : : : : : +- CometBroadcastHashJoin - : : : : : : :- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : : : +- ReusedSubquery - : : : : : : +- CometBroadcastExchange - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : : +- CometBroadcastExchange - : : : : : +- 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 - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : : +- ReusedSubquery - : : : : +- CometBroadcastExchange - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : +- CometBroadcastExchange - : : : +- 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 - : :- CometFilter - : : : +- ReusedSubquery - : : +- CometHashAggregate - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : +- ReusedSubquery - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : +- CometBroadcastExchange - : : : : +- 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 - : : : : : : +- CometBroadcastHashJoin - : : : : : : :- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : : : +- CometBroadcastExchange - : : : : : : +- CometProject - : : : : : : +- CometBroadcastHashJoin - : : : : : : :- CometProject - : : : : : : : +- CometBroadcastHashJoin - : : : : : : : :- CometFilter - : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : : : : +- ReusedSubquery - : : : : : : : +- CometBroadcastExchange - : : : : : : : +- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : : : +- CometBroadcastExchange - : : : : : : +- 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 - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : : : +- ReusedSubquery - : : : : : +- CometBroadcastExchange - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : +- CometBroadcastExchange - : : : +- 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 - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometProject - : : : : : : +- CometBroadcastHashJoin - : : : : : : :- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : : : +- ReusedSubquery - : : : : : : +- CometBroadcastExchange - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : : +- CometBroadcastExchange - : : : : : +- 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 - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : : +- ReusedSubquery - : : : : +- CometBroadcastExchange - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : +- CometBroadcastExchange - : : : +- 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 - : +- CometFilter - : : +- ReusedSubquery - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : +- ReusedSubquery - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : +- CometBroadcastExchange - : : : +- 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 - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometProject - : : : : : : +- CometBroadcastHashJoin - : : : : : : :- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : : : +- ReusedSubquery - : : : : : : +- CometBroadcastExchange - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : : +- CometBroadcastExchange - : : : : : +- 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 - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : : +- ReusedSubquery - : : : : +- CometBroadcastExchange - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : +- CometBroadcastExchange - : : +- 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 - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : : +- ReusedSubquery - : : : : : +- CometBroadcastExchange - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : +- CometBroadcastExchange - : : : : +- 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 - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : +- ReusedSubquery - : : : +- CometBroadcastExchange - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : +- CometBroadcastExchange - : : +- 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 - :- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometHashAggregate - : +- CometHashAggregate - : +- CometUnion - : :- CometFilter - : : : +- ReusedSubquery - : : +- CometHashAggregate - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- 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 - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : +- CometBroadcastExchange - : : : : +- 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 - : : : : : : +- CometBroadcastHashJoin - : : : : : : :- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : : : +- CometBroadcastExchange - : : : : : : +- CometProject - : : : : : : +- CometBroadcastHashJoin - : : : : : : :- CometProject - : : : : : : : +- CometBroadcastHashJoin - : : : : : : : :- CometFilter - : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : : : : +- ReusedSubquery - : : : : : : : +- CometBroadcastExchange - : : : : : : : +- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : : : +- CometBroadcastExchange - : : : : : : +- 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 - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : : : +- ReusedSubquery - : : : : : +- CometBroadcastExchange - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : +- CometBroadcastExchange - : : : +- 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 - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometProject - : : : : : : +- CometBroadcastHashJoin - : : : : : : :- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : : : +- ReusedSubquery - : : : : : : +- CometBroadcastExchange - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : : +- CometBroadcastExchange - : : : : : +- 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 - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : : +- ReusedSubquery - : : : : +- CometBroadcastExchange - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : +- CometBroadcastExchange - : : : +- 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 - : :- CometFilter - : : : +- ReusedSubquery - : : +- CometHashAggregate - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : +- ReusedSubquery - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : +- CometBroadcastExchange - : : : : +- 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 - : : : : : : +- CometBroadcastHashJoin - : : : : : : :- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : : : +- CometBroadcastExchange - : : : : : : +- CometProject - : : : : : : +- CometBroadcastHashJoin - : : : : : : :- CometProject - : : : : : : : +- CometBroadcastHashJoin - : : : : : : : :- CometFilter - : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : : : : +- ReusedSubquery - : : : : : : : +- CometBroadcastExchange - : : : : : : : +- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : : : +- CometBroadcastExchange - : : : : : : +- 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 - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : : : +- ReusedSubquery - : : : : : +- CometBroadcastExchange - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : +- CometBroadcastExchange - : : : +- 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 - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometProject - : : : : : : +- CometBroadcastHashJoin - : : : : : : :- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : : : +- ReusedSubquery - : : : : : : +- CometBroadcastExchange - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : : +- CometBroadcastExchange - : : : : : +- 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 - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : : +- ReusedSubquery - : : : : +- CometBroadcastExchange - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : +- CometBroadcastExchange - : : : +- 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 - : +- CometFilter - : : +- ReusedSubquery - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : +- ReusedSubquery - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : +- CometBroadcastExchange - : : : +- 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 - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometProject - : : : : : : +- CometBroadcastHashJoin - : : : : : : :- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : : : +- ReusedSubquery - : : : : : : +- CometBroadcastExchange - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : : +- CometBroadcastExchange - : : : : : +- 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 - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : : +- ReusedSubquery - : : : : +- CometBroadcastExchange - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : +- CometBroadcastExchange - : : +- 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 - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : : +- ReusedSubquery - : : : : : +- CometBroadcastExchange - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : +- CometBroadcastExchange - : : : : +- 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 - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : +- ReusedSubquery - : : : +- CometBroadcastExchange - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : +- CometBroadcastExchange - : : +- 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 - :- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometHashAggregate - : +- CometHashAggregate - : +- CometUnion - : :- CometFilter - : : : +- ReusedSubquery - : : +- CometHashAggregate - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- 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 - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : +- CometBroadcastExchange - : : : : +- 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 - : : : : : : +- CometBroadcastHashJoin - : : : : : : :- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : : : +- CometBroadcastExchange - : : : : : : +- CometProject - : : : : : : +- CometBroadcastHashJoin - : : : : : : :- CometProject - : : : : : : : +- CometBroadcastHashJoin - : : : : : : : :- CometFilter - : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : : : : +- ReusedSubquery - : : : : : : : +- CometBroadcastExchange - : : : : : : : +- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : : : +- CometBroadcastExchange - : : : : : : +- 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 - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : : : +- ReusedSubquery - : : : : : +- CometBroadcastExchange - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : +- CometBroadcastExchange - : : : +- 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 - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometProject - : : : : : : +- CometBroadcastHashJoin - : : : : : : :- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : : : +- ReusedSubquery - : : : : : : +- CometBroadcastExchange - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : : +- CometBroadcastExchange - : : : : : +- 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 - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : : +- ReusedSubquery - : : : : +- CometBroadcastExchange - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : +- CometBroadcastExchange - : : : +- 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 - : :- CometFilter - : : : +- ReusedSubquery - : : +- CometHashAggregate - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : +- ReusedSubquery - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : +- CometBroadcastExchange - : : : : +- 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 - : : : : : : +- CometBroadcastHashJoin - : : : : : : :- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : : : +- CometBroadcastExchange - : : : : : : +- CometProject - : : : : : : +- CometBroadcastHashJoin - : : : : : : :- CometProject - : : : : : : : +- CometBroadcastHashJoin - : : : : : : : :- CometFilter - : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : : : : +- ReusedSubquery - : : : : : : : +- CometBroadcastExchange - : : : : : : : +- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : : : +- CometBroadcastExchange - : : : : : : +- 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 - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : : : +- ReusedSubquery - : : : : : +- CometBroadcastExchange - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : +- CometBroadcastExchange - : : : +- 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 - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometProject - : : : : : : +- CometBroadcastHashJoin - : : : : : : :- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : : : +- ReusedSubquery - : : : : : : +- CometBroadcastExchange - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : : +- CometBroadcastExchange - : : : : : +- 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 - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : : +- ReusedSubquery - : : : : +- CometBroadcastExchange - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : +- CometBroadcastExchange - : : : +- 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 - : +- CometFilter - : : +- ReusedSubquery - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : +- ReusedSubquery - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : +- CometBroadcastExchange - : : : +- 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 - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometProject - : : : : : : +- CometBroadcastHashJoin - : : : : : : :- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : : : +- ReusedSubquery - : : : : : : +- CometBroadcastExchange - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : : +- CometBroadcastExchange - : : : : : +- 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 - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : : +- ReusedSubquery - : : : : +- CometBroadcastExchange - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : +- CometBroadcastExchange - : : +- 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 - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : : +- ReusedSubquery - : : : : : +- CometBroadcastExchange - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : +- CometBroadcastExchange - : : : : +- 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 - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : +- ReusedSubquery - : : : +- CometBroadcastExchange - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : +- CometBroadcastExchange - : : +- 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 - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometHashAggregate - +- CometHashAggregate - +- CometUnion - :- CometFilter - : : +- ReusedSubquery - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- 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 - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : +- CometBroadcastExchange - : : : +- 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 - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometProject - : : : : : : +- CometBroadcastHashJoin - : : : : : : :- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : : : +- ReusedSubquery - : : : : : : +- CometBroadcastExchange - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : : +- CometBroadcastExchange - : : : : : +- 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 - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : : +- ReusedSubquery - : : : : +- CometBroadcastExchange - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : +- CometBroadcastExchange - : : +- 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 - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : : +- ReusedSubquery - : : : : : +- CometBroadcastExchange - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : +- CometBroadcastExchange - : : : : +- 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 - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : +- ReusedSubquery - : : : +- CometBroadcastExchange - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : +- CometBroadcastExchange - : : +- 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 - :- CometFilter - : : +- ReusedSubquery - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : +- ReusedSubquery - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : +- CometBroadcastExchange - : : : +- 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 - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometProject - : : : : : : +- CometBroadcastHashJoin - : : : : : : :- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : : : +- ReusedSubquery - : : : : : : +- CometBroadcastExchange - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : : +- CometBroadcastExchange - : : : : : +- 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 - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : : +- ReusedSubquery - : : : : +- CometBroadcastExchange - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : +- CometBroadcastExchange - : : +- 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 - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : : +- ReusedSubquery - : : : : : +- CometBroadcastExchange - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : +- CometBroadcastExchange - : : : : +- 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 - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : +- ReusedSubquery - : : : +- CometBroadcastExchange - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : +- CometBroadcastExchange - : : +- 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 - +- CometFilter - : +- ReusedSubquery - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : +- ReusedSubquery - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : +- CometBroadcastExchange - : : +- 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 - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : : +- ReusedSubquery - : : : : : +- CometBroadcastExchange - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : +- CometBroadcastExchange - : : : : +- 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 - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : +- ReusedSubquery - : : : +- CometBroadcastExchange - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometBroadcastHashJoin - : :- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : +- CometBroadcastExchange - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : +- CometBroadcastExchange - : +- 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 - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : +- ReusedSubquery - : : : : +- CometBroadcastExchange - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : +- CometBroadcastExchange - : : : +- 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 - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : +- ReusedSubquery - : : +- CometBroadcastExchange - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : +- CometBroadcastExchange - : +- 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 - -Comet accelerated 2010 out of 2169 eligible operators (92%). Final plan contains 38 transitions 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_2/q18a.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_2/q18a.native_datafusion/extended.txt deleted file mode 100644 index 75f132f6fb..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_2/q18a.native_datafusion/extended.txt +++ /dev/null @@ -1,213 +0,0 @@ -CometNativeColumnarToRow -+- CometTakeOrderedAndProject - +- CometUnion - :- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometProject - : : : : : : +- CometBroadcastHashJoin - : : : : : : :- CometFilter - : : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : : : : +- CometSubqueryBroadcast - : : : : : : : +- CometBroadcastExchange - : : : : : : : +- CometProject - : : : : : : : +- CometFilter - : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : : : +- CometBroadcastExchange - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.customer_demographics - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.customer - : : : : +- CometBroadcastExchange - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.customer_demographics - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.customer_address - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.item - :- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometProject - : : : : : : +- CometBroadcastHashJoin - : : : : : : :- CometFilter - : : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : : : : +- ReusedSubquery - : : : : : : +- CometBroadcastExchange - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.customer_demographics - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.customer - : : : : +- CometBroadcastExchange - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.customer_demographics - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.customer_address - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.item - :- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometProject - : : : : : : +- CometBroadcastHashJoin - : : : : : : :- CometFilter - : : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : : : : +- ReusedSubquery - : : : : : : +- CometBroadcastExchange - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.customer_demographics - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.customer - : : : : +- CometBroadcastExchange - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.customer_demographics - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.customer_address - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.item - :- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometProject - : : : : : : +- CometBroadcastHashJoin - : : : : : : :- CometFilter - : : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : : : : +- ReusedSubquery - : : : : : : +- CometBroadcastExchange - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.customer_demographics - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.customer - : : : : +- CometBroadcastExchange - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.customer_demographics - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.customer_address - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.item - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : : : +- ReusedSubquery - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.customer_demographics - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.customer - : : : +- CometBroadcastExchange - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.customer_demographics - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.customer_address - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.date_dim - +- CometBroadcastExchange - +- CometFilter - +- CometNativeScan parquet spark_catalog.default.item - -Comet accelerated 205 out of 210 eligible operators (97%). Final plan contains 1 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_2/q18a.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_2/q18a.native_iceberg_compat/extended.txt deleted file mode 100644 index 94f7834f85..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_2/q18a.native_iceberg_compat/extended.txt +++ /dev/null @@ -1,214 +0,0 @@ -CometNativeColumnarToRow -+- CometTakeOrderedAndProject - +- CometUnion - :- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometProject - : : : : : : +- CometBroadcastHashJoin - : : : : : : :- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_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.customer_demographics - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer - : : : : +- CometBroadcastExchange - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics - : : : +- 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 - :- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- 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.customer_demographics - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer - : : : : +- CometBroadcastExchange - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics - : : : +- 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 - :- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- 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.customer_demographics - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer - : : : : +- CometBroadcastExchange - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics - : : : +- 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 - :- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- 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.customer_demographics - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer - : : : : +- CometBroadcastExchange - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics - : : : +- 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 - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- 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.customer_demographics - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer - : : : +- CometBroadcastExchange - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics - : : +- 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 - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - -Comet accelerated 204 out of 210 eligible operators (97%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_2/q20.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_2/q20.native_datafusion/extended.txt deleted file mode 100644 index 24498162ca..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_2/q20.native_datafusion/extended.txt +++ /dev/null @@ -1,30 +0,0 @@ -TakeOrderedAndProject -+- 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 - :- CometProject - : +- CometBroadcastHashJoin - : :- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : +- CometSubqueryBroadcast - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.item - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometNativeScan parquet spark_catalog.default.date_dim - -Comet accelerated 23 out of 27 eligible operators (85%). Final plan contains 1 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_2/q20.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_2/q20.native_iceberg_compat/extended.txt deleted file mode 100644 index 643e47b0c6..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_2/q20.native_iceberg_compat/extended.txt +++ /dev/null @@ -1,31 +0,0 @@ -TakeOrderedAndProject -+- 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 - :- CometProject - : +- CometBroadcastHashJoin - : :- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_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.item - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - -Comet accelerated 22 out of 27 eligible operators (81%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_2/q22.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_2/q22.native_datafusion/extended.txt deleted file mode 100644 index d669223566..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_2/q22.native_datafusion/extended.txt +++ /dev/null @@ -1,32 +0,0 @@ -TakeOrderedAndProject -+- HashAggregate - +- Exchange - +- HashAggregate - +- Expand - +- Project - +- BroadcastNestedLoopJoin [COMET: BroadcastNestedLoopJoin is not supported] - :- CometNativeColumnarToRow - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.inventory - : : : +- CometSubqueryBroadcast - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.item - +- BroadcastExchange - +- CometNativeColumnarToRow - +- CometNativeScan parquet spark_catalog.default.warehouse - -Comet accelerated 19 out of 28 eligible operators (67%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_2/q22.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_2/q22.native_iceberg_compat/extended.txt deleted file mode 100644 index 136315f3db..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_2/q22.native_iceberg_compat/extended.txt +++ /dev/null @@ -1,33 +0,0 @@ -TakeOrderedAndProject -+- HashAggregate - +- 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 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_2/q22a.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_2/q22a.native_datafusion/extended.txt deleted file mode 100644 index 0d8c15e00c..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_2/q22a.native_datafusion/extended.txt +++ /dev/null @@ -1,154 +0,0 @@ -CometNativeColumnarToRow -+- CometTakeOrderedAndProject - +- CometUnion - :- CometHashAggregate - : +- CometHashAggregate - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.inventory - : : : : +- CometSubqueryBroadcast - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.item - : +- CometBroadcastExchange - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.warehouse - :- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.inventory - : : : : +- CometSubqueryBroadcast - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.item - : +- CometBroadcastExchange - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.warehouse - :- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.inventory - : : : : +- CometSubqueryBroadcast - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.item - : +- CometBroadcastExchange - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.warehouse - :- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.inventory - : : : : +- CometSubqueryBroadcast - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.item - : +- CometBroadcastExchange - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.warehouse - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.inventory - : : : +- CometSubqueryBroadcast - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.item - +- CometBroadcastExchange - +- CometFilter - +- CometNativeScan parquet spark_catalog.default.warehouse - -Comet accelerated 146 out of 151 eligible operators (96%). Final plan contains 1 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_2/q22a.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_2/q22a.native_iceberg_compat/extended.txt deleted file mode 100644 index d83496133a..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_2/q22a.native_iceberg_compat/extended.txt +++ /dev/null @@ -1,159 +0,0 @@ -CometNativeColumnarToRow -+- CometTakeOrderedAndProject - +- CometUnion - :- CometHashAggregate - : +- CometHashAggregate - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- 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 - : +- CometBroadcastExchange - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.warehouse - :- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- 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 - : +- CometBroadcastExchange - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.warehouse - :- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- 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 - : +- CometBroadcastExchange - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.warehouse - :- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- 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 - : +- CometBroadcastExchange - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.warehouse - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- 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 - +- CometBroadcastExchange - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.warehouse - -Comet accelerated 141 out of 151 eligible operators (93%). Final plan contains 6 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_2/q24.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_2/q24.native_datafusion/extended.txt deleted file mode 100644 index 8e67cf3e18..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_2/q24.native_datafusion/extended.txt +++ /dev/null @@ -1,95 +0,0 @@ -CometNativeColumnarToRow -+- CometSort - +- CometColumnarExchange - +- Filter - : +- Subquery - : +- HashAggregate - : +- Exchange - : +- HashAggregate - : +- HashAggregate - : +- 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 - +- Exchange - +- HashAggregate - +- HashAggregate - +- 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 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_2/q24.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_2/q24.native_iceberg_compat/extended.txt deleted file mode 100644 index 4488b9b0ff..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_2/q24.native_iceberg_compat/extended.txt +++ /dev/null @@ -1,95 +0,0 @@ -CometNativeColumnarToRow -+- CometSort - +- CometColumnarExchange - +- Filter - : +- Subquery - : +- HashAggregate - : +- Exchange - : +- HashAggregate - : +- HashAggregate - : +- 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 - +- Exchange - +- HashAggregate - +- HashAggregate - +- 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 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_2/q27a.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_2/q27a.native_datafusion/extended.txt deleted file mode 100644 index 70614792ba..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_2/q27a.native_datafusion/extended.txt +++ /dev/null @@ -1,98 +0,0 @@ -CometNativeColumnarToRow -+- CometTakeOrderedAndProject - +- CometUnion - :- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : : : +- CometSubqueryBroadcast - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.customer_demographics - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.store - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.item - :- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : : : +- ReusedSubquery - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.customer_demographics - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.store - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.item - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : : +- ReusedSubquery - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.customer_demographics - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.store - +- CometBroadcastExchange - +- CometFilter - +- CometNativeScan parquet spark_catalog.default.item - -Comet accelerated 92 out of 95 eligible operators (96%). Final plan contains 1 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_2/q27a.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_2/q27a.native_iceberg_compat/extended.txt deleted file mode 100644 index e7288e7e47..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_2/q27a.native_iceberg_compat/extended.txt +++ /dev/null @@ -1,99 +0,0 @@ -CometNativeColumnarToRow -+- CometTakeOrderedAndProject - +- CometUnion - :- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- 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.customer_demographics - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - :- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- 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.customer_demographics - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- 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.customer_demographics - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - +- CometBroadcastExchange - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - -Comet accelerated 91 out of 95 eligible operators (95%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_2/q34.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_2/q34.native_datafusion/extended.txt deleted file mode 100644 index 4c0ef9abcd..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_2/q34.native_datafusion/extended.txt +++ /dev/null @@ -1,40 +0,0 @@ -CometNativeColumnarToRow -+- CometSort - +- CometExchange - +- CometProject - +- CometBroadcastHashJoin - :- CometFilter - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : : +- CometSubqueryBroadcast - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.store - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.household_demographics - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometNativeScan parquet spark_catalog.default.customer - -Comet accelerated 36 out of 37 eligible operators (97%). Final plan contains 1 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_2/q34.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_2/q34.native_iceberg_compat/extended.txt deleted file mode 100644 index 18ed3e6335..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_2/q34.native_iceberg_compat/extended.txt +++ /dev/null @@ -1,41 +0,0 @@ -CometNativeColumnarToRow -+- CometSort - +- CometExchange - +- CometProject - +- CometBroadcastHashJoin - :- CometFilter - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- 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.store - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.household_demographics - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer - -Comet accelerated 35 out of 37 eligible operators (94%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_2/q35.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_2/q35.native_datafusion/extended.txt deleted file mode 100644 index 5e8a5364b6..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_2/q35.native_datafusion/extended.txt +++ /dev/null @@ -1,61 +0,0 @@ -TakeOrderedAndProject -+- HashAggregate - +- Exchange - +- HashAggregate - +- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- Project - : : +- Filter - : : +- BroadcastHashJoin - : : :- BroadcastHashJoin [COMET: Unsupported join type ExistenceJoin(exists#1)] - : : : :- CometNativeColumnarToRow - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.customer - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometNativeScan parquet spark_catalog.default.store_sales - : : : : : +- CometSubqueryBroadcast - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometNativeScan parquet spark_catalog.default.web_sales - : : : : +- ReusedSubquery - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : +- ReusedSubquery - : : +- CometBroadcastExchange - : : +- 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 35 out of 54 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-v2_7-spark4_2/q35.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_2/q35.native_iceberg_compat/extended.txt deleted file mode 100644 index 72960fc267..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_2/q35.native_iceberg_compat/extended.txt +++ /dev/null @@ -1,62 +0,0 @@ -TakeOrderedAndProject -+- HashAggregate - +- 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 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_2/q35a.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_2/q35a.native_datafusion/extended.txt deleted file mode 100644 index f27de2fdbb..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_2/q35a.native_datafusion/extended.txt +++ /dev/null @@ -1,55 +0,0 @@ -CometNativeColumnarToRow -+- CometTakeOrderedAndProject - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.customer - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometNativeScan parquet spark_catalog.default.store_sales - : : : : +- CometSubqueryBroadcast - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometUnion - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometNativeScan parquet spark_catalog.default.web_sales - : : : : +- ReusedSubquery - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : +- ReusedSubquery - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.customer_address - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometNativeScan parquet spark_catalog.default.customer_demographics - -Comet accelerated 49 out of 52 eligible operators (94%). Final plan contains 1 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_2/q35a.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_2/q35a.native_iceberg_compat/extended.txt deleted file mode 100644 index bdbfb0d6dc..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_2/q35a.native_iceberg_compat/extended.txt +++ /dev/null @@ -1,56 +0,0 @@ -CometNativeColumnarToRow -+- CometTakeOrderedAndProject - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- 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 - : : +- CometBroadcastExchange - : : +- CometUnion - : : :- 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 - : : +- 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 - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics - -Comet accelerated 48 out of 52 eligible operators (92%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_2/q36a.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_2/q36a.native_datafusion/extended.txt deleted file mode 100644 index be0d09e04e..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_2/q36a.native_datafusion/extended.txt +++ /dev/null @@ -1,102 +0,0 @@ -TakeOrderedAndProject -+- 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 - +- CometUnion - :- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : : +- CometSubqueryBroadcast - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.item - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.store - :- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : : +- CometSubqueryBroadcast - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.item - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.store - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : +- CometSubqueryBroadcast - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.item - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometNativeScan parquet spark_catalog.default.store - -Comet accelerated 93 out of 99 eligible operators (93%). Final plan contains 1 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_2/q36a.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_2/q36a.native_iceberg_compat/extended.txt deleted file mode 100644 index e9571a1f12..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_2/q36a.native_iceberg_compat/extended.txt +++ /dev/null @@ -1,105 +0,0 @@ -TakeOrderedAndProject -+- 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 - +- CometUnion - :- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- 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.item - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - :- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- 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.item - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- 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.item - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - -Comet accelerated 90 out of 99 eligible operators (90%). Final plan contains 4 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_2/q47.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_2/q47.native_datafusion/extended.txt deleted file mode 100644 index 54fa1daf2e..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_2/q47.native_datafusion/extended.txt +++ /dev/null @@ -1,102 +0,0 @@ -TakeOrderedAndProject -+- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- Project - : : +- Filter - : : +- Window - : : +- 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 - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : +- CometBroadcastExchange - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : : +- CometSubqueryBroadcast - : : : : +- CometBroadcastExchange - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.store - : +- BroadcastExchange - : +- Project - : +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] - : +- CometNativeColumnarToRow - : +- CometSort - : +- CometExchange - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : +- CometBroadcastExchange - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : +- CometSubqueryBroadcast - : : : +- CometBroadcastExchange - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.store - +- BroadcastExchange - +- Project - +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] - +- CometNativeColumnarToRow - +- CometSort - +- CometExchange - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.item - : : +- CometBroadcastExchange - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : +- CometSubqueryBroadcast - : : +- CometBroadcastExchange - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.date_dim - +- CometBroadcastExchange - +- CometFilter - +- CometNativeScan parquet spark_catalog.default.store - -Comet accelerated 78 out of 97 eligible operators (80%). Final plan contains 3 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_2/q47.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_2/q47.native_iceberg_compat/extended.txt deleted file mode 100644 index 9691120d2f..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_2/q47.native_iceberg_compat/extended.txt +++ /dev/null @@ -1,105 +0,0 @@ -TakeOrderedAndProject -+- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- Project - : : +- Filter - : : +- Window - : : +- 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 - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : +- CometBroadcastExchange - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - : +- BroadcastExchange - : +- Project - : +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] - : +- CometNativeColumnarToRow - : +- CometSort - : +- CometExchange - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : +- CometBroadcastExchange - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - +- BroadcastExchange - +- Project - +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] - +- CometNativeColumnarToRow - +- CometSort - +- CometExchange - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : +- CometBroadcastExchange - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - +- CometBroadcastExchange - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - -Comet accelerated 75 out of 97 eligible operators (77%). Final plan contains 6 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_2/q49.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_2/q49.native_datafusion/extended.txt deleted file mode 100644 index 138d841ed0..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_2/q49.native_datafusion/extended.txt +++ /dev/null @@ -1,91 +0,0 @@ -TakeOrderedAndProject -+- HashAggregate - +- HashAggregate - +- Union - :- Project - : +- Filter - : +- Window - : +- Sort - : +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set 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 - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : +- CometSubqueryBroadcast - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.web_returns - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.date_dim - :- Project - : +- Filter - : +- Window - : +- Sort - : +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set 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 - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : +- ReusedSubquery - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.catalog_returns - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.date_dim - +- Project - +- Filter - +- Window - +- Sort - +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set 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 - :- CometProject - : +- CometBroadcastHashJoin - : :- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : +- ReusedSubquery - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.store_returns - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometNativeScan parquet spark_catalog.default.date_dim - -Comet accelerated 64 out of 86 eligible operators (74%). 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_2/q49.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_2/q49.native_iceberg_compat/extended.txt deleted file mode 100644 index 2044df0e8a..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_2/q49.native_iceberg_compat/extended.txt +++ /dev/null @@ -1,92 +0,0 @@ -TakeOrderedAndProject -+- HashAggregate - +- HashAggregate - +- Union - :- Project - : +- Filter - : +- Window - : +- Sort - : +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set 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 - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometBroadcastExchange - : : : +- CometProject - : : : +- 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 - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - :- Project - : +- Filter - : +- Window - : +- Sort - : +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set 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 - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : +- ReusedSubquery - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - +- Project - +- Filter - +- Window - +- Sort - +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set 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 - :- CometProject - : +- CometBroadcastHashJoin - : :- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : +- ReusedSubquery - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - -Comet accelerated 63 out of 86 eligible operators (73%). Final plan contains 4 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_2/q51a.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_2/q51a.native_datafusion/extended.txt deleted file mode 100644 index 36a797bad9..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_2/q51a.native_datafusion/extended.txt +++ /dev/null @@ -1,208 +0,0 @@ -TakeOrderedAndProject -+- Filter - +- HashAggregate - +- HashAggregate - +- Project - +- BroadcastHashJoin - :- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set 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 - : +- CometProject - : +- CometFilter - : +- CometSortMergeJoin - : :- CometSort - : : +- CometColumnarExchange - : : +- HashAggregate - : : +- 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 - : : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : : +- CometSubqueryBroadcast - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- Project - : : +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] - : : +- CometNativeColumnarToRow - : : +- CometSort - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : +- CometSubqueryBroadcast - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- CometSort - : +- CometColumnarExchange - : +- HashAggregate - : +- 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 - : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : +- ReusedSubquery - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- Project - : +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] - : +- CometNativeColumnarToRow - : +- CometSort - : +- CometExchange - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : +- ReusedSubquery - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.date_dim - +- BroadcastExchange - +- Project - +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] - +- CometNativeColumnarToRow - +- CometSort - +- CometExchange - +- CometProject - +- CometFilter - +- CometSortMergeJoin - :- CometSort - : +- CometColumnarExchange - : +- HashAggregate - : +- 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 - : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : +- CometSubqueryBroadcast - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- Project - : +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] - : +- CometNativeColumnarToRow - : +- CometSort - : +- CometExchange - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : +- CometSubqueryBroadcast - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.date_dim - +- CometSort - +- CometColumnarExchange - +- HashAggregate - +- 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 - : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : +- ReusedSubquery - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.date_dim - +- BroadcastExchange - +- Project - +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] - +- CometNativeColumnarToRow - +- CometSort - +- CometExchange - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometFilter - : +- CometNativeScan parquet spark_catalog.default.store_sales - : +- ReusedSubquery - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometNativeScan parquet spark_catalog.default.date_dim - -Comet accelerated 138 out of 196 eligible operators (70%). 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_2/q51a.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_2/q51a.native_iceberg_compat/extended.txt deleted file mode 100644 index 190830204f..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_2/q51a.native_iceberg_compat/extended.txt +++ /dev/null @@ -1,212 +0,0 @@ -TakeOrderedAndProject -+- Filter - +- HashAggregate - +- HashAggregate - +- Project - +- BroadcastHashJoin - :- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set 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 - : +- CometProject - : +- CometFilter - : +- CometSortMergeJoin - : :- CometSort - : : +- CometColumnarExchange - : : +- HashAggregate - : : +- 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 - : +- 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).] - +- CometNativeColumnarToRow - +- CometSort - +- CometExchange - +- CometProject - +- CometFilter - +- CometSortMergeJoin - :- CometSort - : +- CometColumnarExchange - : +- HashAggregate - : +- 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 - +- 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 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_2/q57.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_2/q57.native_datafusion/extended.txt deleted file mode 100644 index aa4ccf3246..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_2/q57.native_datafusion/extended.txt +++ /dev/null @@ -1,102 +0,0 @@ -TakeOrderedAndProject -+- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- Project - : : +- Filter - : : +- Window - : : +- 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 - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : +- CometBroadcastExchange - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : +- CometSubqueryBroadcast - : : : : +- CometBroadcastExchange - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.call_center - : +- BroadcastExchange - : +- Project - : +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] - : +- CometNativeColumnarToRow - : +- CometSort - : +- CometExchange - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : +- CometBroadcastExchange - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : +- CometSubqueryBroadcast - : : : +- CometBroadcastExchange - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.call_center - +- BroadcastExchange - +- Project - +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] - +- CometNativeColumnarToRow - +- CometSort - +- CometExchange - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.item - : : +- CometBroadcastExchange - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : +- CometSubqueryBroadcast - : : +- CometBroadcastExchange - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.date_dim - +- CometBroadcastExchange - +- CometFilter - +- CometNativeScan parquet spark_catalog.default.call_center - -Comet accelerated 78 out of 97 eligible operators (80%). Final plan contains 3 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_2/q57.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_2/q57.native_iceberg_compat/extended.txt deleted file mode 100644 index 5e8acff889..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_2/q57.native_iceberg_compat/extended.txt +++ /dev/null @@ -1,105 +0,0 @@ -TakeOrderedAndProject -+- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- Project - : : +- Filter - : : +- Window - : : +- 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 - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : +- CometBroadcastExchange - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.call_center - : +- BroadcastExchange - : +- Project - : +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] - : +- CometNativeColumnarToRow - : +- CometSort - : +- CometExchange - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : +- CometBroadcastExchange - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.call_center - +- BroadcastExchange - +- Project - +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] - +- CometNativeColumnarToRow - +- CometSort - +- CometExchange - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : +- CometBroadcastExchange - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - +- CometBroadcastExchange - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.call_center - -Comet accelerated 75 out of 97 eligible operators (77%). Final plan contains 6 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_2/q6.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_2/q6.native_datafusion/extended.txt deleted file mode 100644 index dd68f2476e..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_2/q6.native_datafusion/extended.txt +++ /dev/null @@ -1,65 +0,0 @@ -CometNativeColumnarToRow -+- CometTakeOrderedAndProject - +- CometFilter - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.customer_address - : : : +- CometBroadcastExchange - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.customer - : : +- CometBroadcastExchange - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : +- CometSubqueryBroadcast - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : : +- ReusedSubquery - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- Subquery - : : +- CometNativeColumnarToRow - : : +- CometHashAggregate - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : : +- ReusedSubquery - : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- Subquery - : +- CometNativeColumnarToRow - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.date_dim - +- CometBroadcastExchange - +- CometProject - +- CometBroadcastHashJoin - :- CometFilter - : +- CometNativeScan parquet spark_catalog.default.item - +- CometBroadcastExchange - +- CometFilter - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometFilter - +- CometNativeScan parquet spark_catalog.default.item - -Comet accelerated 55 out of 60 eligible operators (91%). Final plan contains 3 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_2/q6.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_2/q6.native_iceberg_compat/extended.txt deleted file mode 100644 index ee55822981..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_2/q6.native_iceberg_compat/extended.txt +++ /dev/null @@ -1,59 +0,0 @@ -CometNativeColumnarToRow -+- CometTakeOrderedAndProject - +- CometFilter - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address - : : : +- CometBroadcastExchange - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer - : : +- CometBroadcastExchange - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometProject - : : +- CometFilter - : : : +- ReusedSubquery - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- Subquery - : : +- CometNativeColumnarToRow - : : +- CometHashAggregate - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : : +- ReusedSubquery - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- ReusedSubquery - +- CometBroadcastExchange - +- CometProject - +- CometBroadcastHashJoin - :- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - +- CometBroadcastExchange - +- CometFilter - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - -Comet accelerated 48 out of 54 eligible operators (88%). 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_2/q64.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_2/q64.native_datafusion/extended.txt deleted file mode 100644 index 14ffe4eab9..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_2/q64.native_datafusion/extended.txt +++ /dev/null @@ -1,245 +0,0 @@ -CometNativeColumnarToRow -+- CometSort - +- CometExchange - +- CometProject - +- CometSortMergeJoin - :- CometSort - : +- CometExchange - : +- CometHashAggregate - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometProject - : : : : : : +- CometBroadcastHashJoin - : : : : : : :- CometProject - : : : : : : : +- CometBroadcastHashJoin - : : : : : : : :- CometProject - : : : : : : : : +- CometBroadcastHashJoin - : : : : : : : : :- CometProject - : : : : : : : : : +- CometBroadcastHashJoin - : : : : : : : : : :- CometProject - : : : : : : : : : : +- CometBroadcastHashJoin - : : : : : : : : : : :- CometProject - : : : : : : : : : : : +- CometBroadcastHashJoin - : : : : : : : : : : : :- CometProject - : : : : : : : : : : : : +- CometBroadcastHashJoin - : : : : : : : : : : : : :- CometProject - : : : : : : : : : : : : : +- CometBroadcastHashJoin - : : : : : : : : : : : : : :- CometProject - : : : : : : : : : : : : : : +- CometBroadcastHashJoin - : : : : : : : : : : : : : : :- CometProject - : : : : : : : : : : : : : : : +- CometBroadcastHashJoin - : : : : : : : : : : : : : : : :- CometProject - : : : : : : : : : : : : : : : : +- CometSortMergeJoin - : : : : : : : : : : : : : : : : :- CometSort - : : : : : : : : : : : : : : : : : +- CometExchange - : : : : : : : : : : : : : : : : : +- CometProject - : : : : : : : : : : : : : : : : : +- CometBroadcastHashJoin - : : : : : : : : : : : : : : : : : :- CometBroadcastExchange - : : : : : : : : : : : : : : : : : : +- CometFilter - : : : : : : : : : : : : : : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : : : : : : : : : : : : : : : : +- CometSubqueryBroadcast - : : : : : : : : : : : : : : : : : : +- CometBroadcastExchange - : : : : : : : : : : : : : : : : : : +- CometFilter - : : : : : : : : : : : : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : : : : : : : : : : : : : : +- CometProject - : : : : : : : : : : : : : : : : : +- CometFilter - : : : : : : : : : : : : : : : : : +- CometNativeScan parquet spark_catalog.default.store_returns - : : : : : : : : : : : : : : : : +- CometSort - : : : : : : : : : : : : : : : : +- CometProject - : : : : : : : : : : : : : : : : +- CometFilter - : : : : : : : : : : : : : : : : +- CometHashAggregate - : : : : : : : : : : : : : : : : +- CometExchange - : : : : : : : : : : : : : : : : +- CometHashAggregate - : : : : : : : : : : : : : : : : +- CometProject - : : : : : : : : : : : : : : : : +- CometSortMergeJoin - : : : : : : : : : : : : : : : : :- CometSort - : : : : : : : : : : : : : : : : : +- CometExchange - : : : : : : : : : : : : : : : : : +- CometProject - : : : : : : : : : : : : : : : : : +- CometFilter - : : : : : : : : : : : : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : : : : : : : : : : : : : +- CometSort - : : : : : : : : : : : : : : : : +- CometExchange - : : : : : : : : : : : : : : : : +- CometProject - : : : : : : : : : : : : : : : : +- CometFilter - : : : : : : : : : : : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_returns - : : : : : : : : : : : : : : : +- CometBroadcastExchange - : : : : : : : : : : : : : : : +- CometFilter - : : : : : : : : : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : : : : : : : : : : : +- CometBroadcastExchange - : : : : : : : : : : : : : : +- CometProject - : : : : : : : : : : : : : : +- CometFilter - : : : : : : : : : : : : : : +- CometNativeScan parquet spark_catalog.default.store - : : : : : : : : : : : : : +- CometBroadcastExchange - : : : : : : : : : : : : : +- CometFilter - : : : : : : : : : : : : : +- CometNativeScan parquet spark_catalog.default.customer - : : : : : : : : : : : : +- CometBroadcastExchange - : : : : : : : : : : : : +- CometFilter - : : : : : : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : : : : : : : : +- CometBroadcastExchange - : : : : : : : : : : : +- CometFilter - : : : : : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : : : : : : : +- CometBroadcastExchange - : : : : : : : : : : +- CometProject - : : : : : : : : : : +- CometFilter - : : : : : : : : : : +- CometNativeScan parquet spark_catalog.default.customer_demographics - : : : : : : : : : +- CometBroadcastExchange - : : : : : : : : : +- CometProject - : : : : : : : : : +- CometFilter - : : : : : : : : : +- CometNativeScan parquet spark_catalog.default.customer_demographics - : : : : : : : : +- CometBroadcastExchange - : : : : : : : : +- CometFilter - : : : : : : : : +- CometNativeScan parquet spark_catalog.default.promotion - : : : : : : : +- CometBroadcastExchange - : : : : : : : +- CometFilter - : : : : : : : +- CometNativeScan parquet spark_catalog.default.household_demographics - : : : : : : +- CometBroadcastExchange - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.household_demographics - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.customer_address - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.customer_address - : : : +- CometBroadcastExchange - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.income_band - : : +- CometBroadcastExchange - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.income_band - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.item - +- CometSort - +- CometExchange - +- CometHashAggregate - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometProject - : : : : : : +- CometBroadcastHashJoin - : : : : : : :- CometProject - : : : : : : : +- CometBroadcastHashJoin - : : : : : : : :- CometProject - : : : : : : : : +- CometBroadcastHashJoin - : : : : : : : : :- CometProject - : : : : : : : : : +- CometBroadcastHashJoin - : : : : : : : : : :- CometProject - : : : : : : : : : : +- CometBroadcastHashJoin - : : : : : : : : : : :- CometProject - : : : : : : : : : : : +- CometBroadcastHashJoin - : : : : : : : : : : : :- CometProject - : : : : : : : : : : : : +- CometBroadcastHashJoin - : : : : : : : : : : : : :- CometProject - : : : : : : : : : : : : : +- CometBroadcastHashJoin - : : : : : : : : : : : : : :- CometProject - : : : : : : : : : : : : : : +- CometBroadcastHashJoin - : : : : : : : : : : : : : : :- CometProject - : : : : : : : : : : : : : : : +- CometSortMergeJoin - : : : : : : : : : : : : : : : :- CometSort - : : : : : : : : : : : : : : : : +- CometExchange - : : : : : : : : : : : : : : : : +- CometProject - : : : : : : : : : : : : : : : : +- CometBroadcastHashJoin - : : : : : : : : : : : : : : : : :- CometBroadcastExchange - : : : : : : : : : : : : : : : : : +- CometFilter - : : : : : : : : : : : : : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : : : : : : : : : : : : : : : +- CometSubqueryBroadcast - : : : : : : : : : : : : : : : : : +- CometBroadcastExchange - : : : : : : : : : : : : : : : : : +- CometFilter - : : : : : : : : : : : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : : : : : : : : : : : : : +- CometProject - : : : : : : : : : : : : : : : : +- CometFilter - : : : : : : : : : : : : : : : : +- CometNativeScan parquet spark_catalog.default.store_returns - : : : : : : : : : : : : : : : +- CometSort - : : : : : : : : : : : : : : : +- CometProject - : : : : : : : : : : : : : : : +- CometFilter - : : : : : : : : : : : : : : : +- CometHashAggregate - : : : : : : : : : : : : : : : +- CometExchange - : : : : : : : : : : : : : : : +- CometHashAggregate - : : : : : : : : : : : : : : : +- CometProject - : : : : : : : : : : : : : : : +- CometSortMergeJoin - : : : : : : : : : : : : : : : :- CometSort - : : : : : : : : : : : : : : : : +- CometExchange - : : : : : : : : : : : : : : : : +- CometProject - : : : : : : : : : : : : : : : : +- CometFilter - : : : : : : : : : : : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : : : : : : : : : : : : +- CometSort - : : : : : : : : : : : : : : : +- CometExchange - : : : : : : : : : : : : : : : +- CometProject - : : : : : : : : : : : : : : : +- CometFilter - : : : : : : : : : : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_returns - : : : : : : : : : : : : : : +- CometBroadcastExchange - : : : : : : : : : : : : : : +- CometFilter - : : : : : : : : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : : : : : : : : : : +- CometBroadcastExchange - : : : : : : : : : : : : : +- CometProject - : : : : : : : : : : : : : +- CometFilter - : : : : : : : : : : : : : +- CometNativeScan parquet spark_catalog.default.store - : : : : : : : : : : : : +- CometBroadcastExchange - : : : : : : : : : : : : +- CometFilter - : : : : : : : : : : : : +- CometNativeScan parquet spark_catalog.default.customer - : : : : : : : : : : : +- CometBroadcastExchange - : : : : : : : : : : : +- CometFilter - : : : : : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : : : : : : : +- CometBroadcastExchange - : : : : : : : : : : +- CometFilter - : : : : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : : : : : : +- CometBroadcastExchange - : : : : : : : : : +- CometProject - : : : : : : : : : +- CometFilter - : : : : : : : : : +- CometNativeScan parquet spark_catalog.default.customer_demographics - : : : : : : : : +- CometBroadcastExchange - : : : : : : : : +- CometProject - : : : : : : : : +- CometFilter - : : : : : : : : +- CometNativeScan parquet spark_catalog.default.customer_demographics - : : : : : : : +- CometBroadcastExchange - : : : : : : : +- CometFilter - : : : : : : : +- CometNativeScan parquet spark_catalog.default.promotion - : : : : : : +- CometBroadcastExchange - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.household_demographics - : : : : : +- CometBroadcastExchange - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.household_demographics - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.customer_address - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.customer_address - : : +- CometBroadcastExchange - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.income_band - : +- CometBroadcastExchange - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.income_band - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometNativeScan parquet spark_catalog.default.item - -Comet accelerated 240 out of 242 eligible operators (99%). Final plan contains 1 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_2/q64.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_2/q64.native_iceberg_compat/extended.txt deleted file mode 100644 index a868670a16..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_2/q64.native_iceberg_compat/extended.txt +++ /dev/null @@ -1,247 +0,0 @@ -CometNativeColumnarToRow -+- CometSort - +- CometExchange - +- CometProject - +- CometSortMergeJoin - :- CometSort - : +- CometExchange - : +- CometHashAggregate - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometProject - : : : : : : +- CometBroadcastHashJoin - : : : : : : :- CometProject - : : : : : : : +- CometBroadcastHashJoin - : : : : : : : :- CometProject - : : : : : : : : +- CometBroadcastHashJoin - : : : : : : : : :- CometProject - : : : : : : : : : +- CometBroadcastHashJoin - : : : : : : : : : :- CometProject - : : : : : : : : : : +- CometBroadcastHashJoin - : : : : : : : : : : :- CometProject - : : : : : : : : : : : +- CometBroadcastHashJoin - : : : : : : : : : : : :- CometProject - : : : : : : : : : : : : +- CometBroadcastHashJoin - : : : : : : : : : : : : :- CometProject - : : : : : : : : : : : : : +- CometBroadcastHashJoin - : : : : : : : : : : : : : :- CometProject - : : : : : : : : : : : : : : +- CometBroadcastHashJoin - : : : : : : : : : : : : : : :- CometProject - : : : : : : : : : : : : : : : +- CometBroadcastHashJoin - : : : : : : : : : : : : : : : :- CometProject - : : : : : : : : : : : : : : : : +- CometSortMergeJoin - : : : : : : : : : : : : : : : : :- CometSort - : : : : : : : : : : : : : : : : : +- CometExchange - : : : : : : : : : : : : : : : : : +- CometProject - : : : : : : : : : : : : : : : : : +- CometBroadcastHashJoin - : : : : : : : : : : : : : : : : : :- CometBroadcastExchange - : : : : : : : : : : : : : : : : : : +- CometFilter - : : : : : : : : : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : : : : : : : : : : : : : : +- SubqueryBroadcast - : : : : : : : : : : : : : : : : : : +- BroadcastExchange - : : : : : : : : : : : : : : : : : : +- CometNativeColumnarToRow - : : : : : : : : : : : : : : : : : : +- CometFilter - : : : : : : : : : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : : : : : : : : : : : : : : +- CometProject - : : : : : : : : : : : : : : : : : +- CometFilter - : : : : : : : : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns - : : : : : : : : : : : : : : : : +- CometSort - : : : : : : : : : : : : : : : : +- CometProject - : : : : : : : : : : : : : : : : +- CometFilter - : : : : : : : : : : : : : : : : +- CometHashAggregate - : : : : : : : : : : : : : : : : +- CometExchange - : : : : : : : : : : : : : : : : +- CometHashAggregate - : : : : : : : : : : : : : : : : +- CometProject - : : : : : : : : : : : : : : : : +- CometSortMergeJoin - : : : : : : : : : : : : : : : : :- CometSort - : : : : : : : : : : : : : : : : : +- CometExchange - : : : : : : : : : : : : : : : : : +- CometProject - : : : : : : : : : : : : : : : : : +- CometFilter - : : : : : : : : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : : : : : : : : : : : : +- CometSort - : : : : : : : : : : : : : : : : +- CometExchange - : : : : : : : : : : : : : : : : +- CometProject - : : : : : : : : : : : : : : : : +- CometFilter - : : : : : : : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns - : : : : : : : : : : : : : : : +- CometBroadcastExchange - : : : : : : : : : : : : : : : +- CometFilter - : : : : : : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : : : : : : : : : : : +- CometBroadcastExchange - : : : : : : : : : : : : : : +- CometProject - : : : : : : : : : : : : : : +- CometFilter - : : : : : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - : : : : : : : : : : : : : +- CometBroadcastExchange - : : : : : : : : : : : : : +- CometFilter - : : : : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer - : : : : : : : : : : : : +- CometBroadcastExchange - : : : : : : : : : : : : +- CometFilter - : : : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : : : : : : : : +- CometBroadcastExchange - : : : : : : : : : : : +- CometFilter - : : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : : : : : : : +- CometBroadcastExchange - : : : : : : : : : : +- CometProject - : : : : : : : : : : +- CometFilter - : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics - : : : : : : : : : +- CometBroadcastExchange - : : : : : : : : : +- CometProject - : : : : : : : : : +- CometFilter - : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics - : : : : : : : : +- CometBroadcastExchange - : : : : : : : : +- CometFilter - : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.promotion - : : : : : : : +- CometBroadcastExchange - : : : : : : : +- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.household_demographics - : : : : : : +- CometBroadcastExchange - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.household_demographics - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address - : : : +- CometBroadcastExchange - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.income_band - : : +- CometBroadcastExchange - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.income_band - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - +- CometSort - +- CometExchange - +- CometHashAggregate - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometProject - : : : : : : +- CometBroadcastHashJoin - : : : : : : :- CometProject - : : : : : : : +- CometBroadcastHashJoin - : : : : : : : :- CometProject - : : : : : : : : +- CometBroadcastHashJoin - : : : : : : : : :- CometProject - : : : : : : : : : +- CometBroadcastHashJoin - : : : : : : : : : :- CometProject - : : : : : : : : : : +- CometBroadcastHashJoin - : : : : : : : : : : :- CometProject - : : : : : : : : : : : +- CometBroadcastHashJoin - : : : : : : : : : : : :- CometProject - : : : : : : : : : : : : +- CometBroadcastHashJoin - : : : : : : : : : : : : :- CometProject - : : : : : : : : : : : : : +- CometBroadcastHashJoin - : : : : : : : : : : : : : :- CometProject - : : : : : : : : : : : : : : +- CometBroadcastHashJoin - : : : : : : : : : : : : : : :- CometProject - : : : : : : : : : : : : : : : +- CometSortMergeJoin - : : : : : : : : : : : : : : : :- CometSort - : : : : : : : : : : : : : : : : +- CometExchange - : : : : : : : : : : : : : : : : +- CometProject - : : : : : : : : : : : : : : : : +- CometBroadcastHashJoin - : : : : : : : : : : : : : : : : :- CometBroadcastExchange - : : : : : : : : : : : : : : : : : +- CometFilter - : : : : : : : : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : : : : : : : : : : : : : +- SubqueryBroadcast - : : : : : : : : : : : : : : : : : +- BroadcastExchange - : : : : : : : : : : : : : : : : : +- CometNativeColumnarToRow - : : : : : : : : : : : : : : : : : +- CometFilter - : : : : : : : : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : : : : : : : : : : : : : +- CometProject - : : : : : : : : : : : : : : : : +- CometFilter - : : : : : : : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns - : : : : : : : : : : : : : : : +- CometSort - : : : : : : : : : : : : : : : +- CometProject - : : : : : : : : : : : : : : : +- CometFilter - : : : : : : : : : : : : : : : +- CometHashAggregate - : : : : : : : : : : : : : : : +- CometExchange - : : : : : : : : : : : : : : : +- CometHashAggregate - : : : : : : : : : : : : : : : +- CometProject - : : : : : : : : : : : : : : : +- CometSortMergeJoin - : : : : : : : : : : : : : : : :- CometSort - : : : : : : : : : : : : : : : : +- CometExchange - : : : : : : : : : : : : : : : : +- CometProject - : : : : : : : : : : : : : : : : +- CometFilter - : : : : : : : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : : : : : : : : : : : +- CometSort - : : : : : : : : : : : : : : : +- CometExchange - : : : : : : : : : : : : : : : +- CometProject - : : : : : : : : : : : : : : : +- CometFilter - : : : : : : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns - : : : : : : : : : : : : : : +- CometBroadcastExchange - : : : : : : : : : : : : : : +- CometFilter - : : : : : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : : : : : : : : : : +- CometBroadcastExchange - : : : : : : : : : : : : : +- CometProject - : : : : : : : : : : : : : +- CometFilter - : : : : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - : : : : : : : : : : : : +- CometBroadcastExchange - : : : : : : : : : : : : +- CometFilter - : : : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer - : : : : : : : : : : : +- CometBroadcastExchange - : : : : : : : : : : : +- CometFilter - : : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : : : : : : : +- CometBroadcastExchange - : : : : : : : : : : +- CometFilter - : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : : : : : : +- CometBroadcastExchange - : : : : : : : : : +- CometProject - : : : : : : : : : +- CometFilter - : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics - : : : : : : : : +- CometBroadcastExchange - : : : : : : : : +- CometProject - : : : : : : : : +- CometFilter - : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics - : : : : : : : +- CometBroadcastExchange - : : : : : : : +- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.promotion - : : : : : : +- CometBroadcastExchange - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.household_demographics - : : : : : +- CometBroadcastExchange - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.household_demographics - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address - : : +- CometBroadcastExchange - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.income_band - : +- CometBroadcastExchange - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.income_band - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - -Comet accelerated 238 out of 242 eligible operators (98%). Final plan contains 3 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_2/q67a.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_2/q67a.native_datafusion/extended.txt deleted file mode 100644 index 8abd1fc15f..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_2/q67a.native_datafusion/extended.txt +++ /dev/null @@ -1,289 +0,0 @@ -TakeOrderedAndProject -+- Filter - +- Window - +- WindowGroupLimit [COMET: WindowGroupLimit is not supported] - +- CometNativeColumnarToRow - +- CometSort - +- CometColumnarExchange - +- WindowGroupLimit [COMET: WindowGroupLimit is not supported] - +- CometNativeColumnarToRow - +- CometSort - +- CometUnion - :- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : : +- CometSubqueryBroadcast - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.store - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.item - :- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : : +- CometSubqueryBroadcast - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.store - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.item - :- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : : +- CometSubqueryBroadcast - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.store - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.item - :- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : : +- CometSubqueryBroadcast - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.store - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.item - :- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : : +- CometSubqueryBroadcast - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.store - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.item - :- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : : +- CometSubqueryBroadcast - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.store - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.item - :- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : : +- CometSubqueryBroadcast - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.store - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.item - :- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : : +- CometSubqueryBroadcast - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.store - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.item - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : +- CometSubqueryBroadcast - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.store - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometNativeScan parquet spark_catalog.default.item - -Comet accelerated 271 out of 285 eligible operators (95%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_2/q67a.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_2/q67a.native_iceberg_compat/extended.txt deleted file mode 100644 index 2afcab203a..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_2/q67a.native_iceberg_compat/extended.txt +++ /dev/null @@ -1,298 +0,0 @@ -TakeOrderedAndProject -+- Filter - +- Window - +- WindowGroupLimit [COMET: WindowGroupLimit is not supported] - +- CometNativeColumnarToRow - +- CometSort - +- CometColumnarExchange - +- WindowGroupLimit [COMET: WindowGroupLimit is not supported] - +- CometNativeColumnarToRow - +- CometSort - +- CometUnion - :- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- 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.store - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - :- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- 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.store - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - :- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- 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.store - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - :- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- 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.store - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - :- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- 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.store - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - :- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- 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.store - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - :- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- 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.store - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - :- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- 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.store - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- 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.store - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - -Comet accelerated 262 out of 285 eligible operators (91%). Final plan contains 11 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_2/q70a.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_2/q70a.native_datafusion/extended.txt deleted file mode 100644 index 44999fda71..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_2/q70a.native_datafusion/extended.txt +++ /dev/null @@ -1,168 +0,0 @@ -TakeOrderedAndProject -+- 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 - +- CometColumnarExchange - +- HashAggregate - +- Union - :- HashAggregate - : +- Exchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- CometNativeColumnarToRow - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : +- CometSubqueryBroadcast - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- 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 [COMET: WindowGroupLimit is not supported] - : +- CometNativeColumnarToRow - : +- CometSort - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : +- ReusedSubquery - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.store - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.date_dim - :- HashAggregate - : +- Exchange - : +- HashAggregate - : +- HashAggregate - : +- Exchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- CometNativeColumnarToRow - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : +- CometSubqueryBroadcast - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- 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 [COMET: WindowGroupLimit is not supported] - : +- CometNativeColumnarToRow - : +- CometSort - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : +- ReusedSubquery - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.store - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.date_dim - +- HashAggregate - +- Exchange - +- HashAggregate - +- HashAggregate - +- Exchange - +- HashAggregate - +- Project - +- BroadcastHashJoin - :- CometNativeColumnarToRow - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : +- CometSubqueryBroadcast - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- 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 [COMET: WindowGroupLimit is not supported] - +- CometNativeColumnarToRow - +- CometSort - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : +- ReusedSubquery - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.store - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometNativeScan parquet spark_catalog.default.date_dim - -Comet accelerated 100 out of 156 eligible operators (64%). 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_2/q70a.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_2/q70a.native_iceberg_compat/extended.txt deleted file mode 100644 index ff6d42786b..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_2/q70a.native_iceberg_compat/extended.txt +++ /dev/null @@ -1,171 +0,0 @@ -TakeOrderedAndProject -+- 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 - +- CometColumnarExchange - +- HashAggregate - +- Union - :- HashAggregate - : +- 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 - : +- Exchange - : +- HashAggregate - : +- HashAggregate - : +- 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 - +- Exchange - +- HashAggregate - +- HashAggregate - +- 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 - -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_2/q72.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_2/q72.native_datafusion/extended.txt deleted file mode 100644 index d443b31d1e..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_2/q72.native_datafusion/extended.txt +++ /dev/null @@ -1,71 +0,0 @@ -CometNativeColumnarToRow -+- CometTakeOrderedAndProject - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometSortMergeJoin - :- CometSort - : +- CometExchange - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometProject - : : : : : : +- CometBroadcastHashJoin - : : : : : : :- CometProject - : : : : : : : +- CometBroadcastHashJoin - : : : : : : : :- CometProject - : : : : : : : : +- CometBroadcastHashJoin - : : : : : : : : :- CometProject - : : : : : : : : : +- CometBroadcastHashJoin - : : : : : : : : : :- CometFilter - : : : : : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : : : : : : : +- CometSubqueryBroadcast - : : : : : : : : : : +- CometBroadcastExchange - : : : : : : : : : : +- CometProject - : : : : : : : : : : +- CometFilter - : : : : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : : : : : : +- CometBroadcastExchange - : : : : : : : : : +- CometFilter - : : : : : : : : : +- CometNativeScan parquet spark_catalog.default.inventory - : : : : : : : : +- CometBroadcastExchange - : : : : : : : : +- CometFilter - : : : : : : : : +- CometNativeScan parquet spark_catalog.default.warehouse - : : : : : : : +- CometBroadcastExchange - : : : : : : : +- CometFilter - : : : : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : : : : +- CometBroadcastExchange - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.customer_demographics - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.household_demographics - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.promotion - +- CometSort - +- CometExchange - +- CometProject - +- CometFilter - +- CometNativeScan parquet spark_catalog.default.catalog_returns - -Comet accelerated 67 out of 68 eligible operators (98%). Final plan contains 1 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_2/q72.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_2/q72.native_iceberg_compat/extended.txt deleted file mode 100644 index 22fde3b02c..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_2/q72.native_iceberg_compat/extended.txt +++ /dev/null @@ -1,72 +0,0 @@ -CometNativeColumnarToRow -+- CometTakeOrderedAndProject - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometSortMergeJoin - :- CometSort - : +- CometExchange - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometProject - : : : : : : +- CometBroadcastHashJoin - : : : : : : :- CometProject - : : : : : : : +- CometBroadcastHashJoin - : : : : : : : :- CometProject - : : : : : : : : +- CometBroadcastHashJoin - : : : : : : : : :- CometProject - : : : : : : : : : +- CometBroadcastHashJoin - : : : : : : : : : :- CometFilter - : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_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.inventory - : : : : : : : : +- CometBroadcastExchange - : : : : : : : : +- CometFilter - : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.warehouse - : : : : : : : +- CometBroadcastExchange - : : : : : : : +- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : : : +- CometBroadcastExchange - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.household_demographics - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.promotion - +- CometSort - +- CometExchange - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns - -Comet accelerated 66 out of 68 eligible operators (97%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_2/q74.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_2/q74.native_datafusion/extended.txt deleted file mode 100644 index cfe634568f..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_2/q74.native_datafusion/extended.txt +++ /dev/null @@ -1,88 +0,0 @@ -CometNativeColumnarToRow -+- CometTakeOrderedAndProject - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometHashAggregate - : : : +- CometExchange - : : : +- CometHashAggregate - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.customer - : : : : +- CometBroadcastExchange - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : : +- CometSubqueryBroadcast - : : : : +- CometBroadcastExchange - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometHashAggregate - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.customer - : : : +- CometBroadcastExchange - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : +- CometSubqueryBroadcast - : : : +- CometBroadcastExchange - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometFilter - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.customer - : : +- CometBroadcastExchange - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : +- ReusedSubquery - : +- CometBroadcastExchange - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.date_dim - +- CometBroadcastExchange - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.customer - : +- CometBroadcastExchange - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.web_sales - : +- ReusedSubquery - +- CometBroadcastExchange - +- CometFilter - +- CometNativeScan parquet spark_catalog.default.date_dim - -Comet accelerated 81 out of 85 eligible operators (95%). Final plan contains 1 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_2/q74.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_2/q74.native_iceberg_compat/extended.txt deleted file mode 100644 index 922864baa1..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_2/q74.native_iceberg_compat/extended.txt +++ /dev/null @@ -1,90 +0,0 @@ -CometNativeColumnarToRow -+- CometTakeOrderedAndProject - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometHashAggregate - : : : +- CometExchange - : : : +- CometHashAggregate - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer - : : : : +- CometBroadcastExchange - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometHashAggregate - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer - : : : +- CometBroadcastExchange - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometFilter - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer - : : +- CometBroadcastExchange - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : +- ReusedSubquery - : +- CometBroadcastExchange - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - +- CometBroadcastExchange - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer - : +- CometBroadcastExchange - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : +- ReusedSubquery - +- CometBroadcastExchange - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - -Comet accelerated 79 out of 85 eligible operators (92%). Final plan contains 3 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_2/q75.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_2/q75.native_datafusion/extended.txt deleted file mode 100644 index bccd23b141..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_2/q75.native_datafusion/extended.txt +++ /dev/null @@ -1,170 +0,0 @@ -CometNativeColumnarToRow -+- CometTakeOrderedAndProject - +- CometProject - +- CometSortMergeJoin - :- CometSort - : +- CometExchange - : +- CometFilter - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometUnion - : :- CometProject - : : +- CometSortMergeJoin - : : :- CometSort - : : : +- CometExchange - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : : +- CometSubqueryBroadcast - : : : : : +- CometBroadcastExchange - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : +- CometBroadcastExchange - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- CometSort - : : +- CometExchange - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.catalog_returns - : :- CometProject - : : +- CometSortMergeJoin - : : :- CometSort - : : : +- CometExchange - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : : : +- ReusedSubquery - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.item - : : : +- CometBroadcastExchange - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- CometSort - : : +- CometExchange - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.store_returns - : +- CometProject - : +- CometSortMergeJoin - : :- CometSort - : : +- CometExchange - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : : +- ReusedSubquery - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.item - : : +- CometBroadcastExchange - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- CometSort - : +- CometExchange - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.web_returns - +- CometSort - +- CometExchange - +- CometFilter - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometUnion - :- CometProject - : +- CometSortMergeJoin - : :- CometSort - : : +- CometExchange - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : +- CometSubqueryBroadcast - : : : : +- CometBroadcastExchange - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.item - : : +- CometBroadcastExchange - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- CometSort - : +- CometExchange - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.catalog_returns - :- CometProject - : +- CometSortMergeJoin - : :- CometSort - : : +- CometExchange - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : : +- ReusedSubquery - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.item - : : +- CometBroadcastExchange - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- CometSort - : +- CometExchange - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.store_returns - +- CometProject - +- CometSortMergeJoin - :- CometSort - : +- CometExchange - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : +- ReusedSubquery - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.item - : +- CometBroadcastExchange - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.date_dim - +- CometSort - +- CometExchange - +- CometProject - +- CometFilter - +- CometNativeScan parquet spark_catalog.default.web_returns - -Comet accelerated 161 out of 167 eligible operators (96%). Final plan contains 1 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_2/q75.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_2/q75.native_iceberg_compat/extended.txt deleted file mode 100644 index ec53af4809..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_2/q75.native_iceberg_compat/extended.txt +++ /dev/null @@ -1,172 +0,0 @@ -CometNativeColumnarToRow -+- CometTakeOrderedAndProject - +- CometProject - +- CometSortMergeJoin - :- CometSort - : +- CometExchange - : +- CometFilter - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometUnion - : :- CometProject - : : +- CometSortMergeJoin - : : :- CometSort - : : : +- CometExchange - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : +- CometBroadcastExchange - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometSort - : : +- CometExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns - : :- CometProject - : : +- CometSortMergeJoin - : : :- CometSort - : : : +- CometExchange - : : : +- 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.item - : : : +- CometBroadcastExchange - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometSort - : : +- CometExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns - : +- CometProject - : +- CometSortMergeJoin - : :- CometSort - : : +- CometExchange - : : +- 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.item - : : +- CometBroadcastExchange - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometSort - : +- CometExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns - +- CometSort - +- CometExchange - +- CometFilter - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometUnion - :- CometProject - : +- CometSortMergeJoin - : :- CometSort - : : +- CometExchange - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : +- CometBroadcastExchange - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometSort - : +- CometExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns - :- CometProject - : +- CometSortMergeJoin - : :- CometSort - : : +- CometExchange - : : +- 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.item - : : +- CometBroadcastExchange - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometSort - : +- CometExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns - +- CometProject - +- CometSortMergeJoin - :- CometSort - : +- CometExchange - : +- 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.item - : +- CometBroadcastExchange - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - +- CometSort - +- CometExchange - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns - -Comet accelerated 159 out of 167 eligible operators (95%). 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_2/q77a.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_2/q77a.native_datafusion/extended.txt deleted file mode 100644 index 45a4b3bb5a..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_2/q77a.native_datafusion/extended.txt +++ /dev/null @@ -1,347 +0,0 @@ -CometNativeColumnarToRow -+- CometTakeOrderedAndProject - +- CometHashAggregate - +- CometColumnarExchange - +- HashAggregate - +- Union - :- HashAggregate - : +- Exchange - : +- HashAggregate - : +- Union - : :- CometNativeColumnarToRow - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometHashAggregate - : : : +- CometExchange - : : : +- CometHashAggregate - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : : : +- CometSubqueryBroadcast - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.store - : : +- CometBroadcastExchange - : : +- CometHashAggregate - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.store_returns - : : : : +- ReusedSubquery - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.store - : :- Project - : : +- BroadcastNestedLoopJoin [COMET: BroadcastNestedLoopJoin is not supported] - : : :- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometHashAggregate - : : : +- CometExchange - : : : +- CometHashAggregate - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : +- ReusedSubquery - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- CometNativeColumnarToRow - : : +- CometHashAggregate - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometNativeScan parquet spark_catalog.default.catalog_returns - : : : +- ReusedSubquery - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- CometNativeColumnarToRow - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometHashAggregate - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : : +- ReusedSubquery - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.web_page - : +- CometBroadcastExchange - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.web_returns - : : : +- ReusedSubquery - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.web_page - :- HashAggregate - : +- Exchange - : +- HashAggregate - : +- HashAggregate - : +- Exchange - : +- HashAggregate - : +- Union - : :- CometNativeColumnarToRow - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometHashAggregate - : : : +- CometExchange - : : : +- CometHashAggregate - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : : : +- CometSubqueryBroadcast - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.store - : : +- CometBroadcastExchange - : : +- CometHashAggregate - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.store_returns - : : : : +- ReusedSubquery - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.store - : :- Project - : : +- BroadcastNestedLoopJoin [COMET: BroadcastNestedLoopJoin is not supported] - : : :- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometHashAggregate - : : : +- CometExchange - : : : +- CometHashAggregate - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : +- ReusedSubquery - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- CometNativeColumnarToRow - : : +- CometHashAggregate - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometNativeScan parquet spark_catalog.default.catalog_returns - : : : +- ReusedSubquery - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- CometNativeColumnarToRow - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometHashAggregate - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : : +- ReusedSubquery - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.web_page - : +- CometBroadcastExchange - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.web_returns - : : : +- ReusedSubquery - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.web_page - +- HashAggregate - +- Exchange - +- HashAggregate - +- HashAggregate - +- Exchange - +- HashAggregate - +- Union - :- CometNativeColumnarToRow - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometHashAggregate - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : : +- CometSubqueryBroadcast - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.store - : +- CometBroadcastExchange - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.store_returns - : : : +- ReusedSubquery - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.store - :- Project - : +- BroadcastNestedLoopJoin [COMET: BroadcastNestedLoopJoin is not supported] - : :- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometHashAggregate - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : +- ReusedSubquery - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- CometNativeColumnarToRow - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometNativeScan parquet spark_catalog.default.catalog_returns - : : +- ReusedSubquery - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.date_dim - +- CometNativeColumnarToRow - +- CometProject - +- CometBroadcastHashJoin - :- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : +- ReusedSubquery - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.web_page - +- CometBroadcastExchange - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.web_returns - : : +- ReusedSubquery - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.date_dim - +- CometBroadcastExchange - +- CometFilter - +- CometNativeScan parquet spark_catalog.default.web_page - -Comet accelerated 285 out of 332 eligible operators (85%). 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_2/q77a.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_2/q77a.native_iceberg_compat/extended.txt deleted file mode 100644 index 82e3d521e5..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_2/q77a.native_iceberg_compat/extended.txt +++ /dev/null @@ -1,350 +0,0 @@ -CometNativeColumnarToRow -+- CometTakeOrderedAndProject - +- CometHashAggregate - +- CometColumnarExchange - +- HashAggregate - +- Union - :- HashAggregate - : +- 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 - : +- Exchange - : +- HashAggregate - : +- HashAggregate - : +- 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 - +- Exchange - +- HashAggregate - +- HashAggregate - +- 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 - -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_2/q78.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_2/q78.native_datafusion/extended.txt deleted file mode 100644 index d29dbc13e5..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_2/q78.native_datafusion/extended.txt +++ /dev/null @@ -1,79 +0,0 @@ -TakeOrderedAndProject -+- Project [COMET: Comet does not support Spark's BigDecimal rounding] - +- CometNativeColumnarToRow - +- CometSortMergeJoin - :- CometProject - : +- CometSortMergeJoin - : :- CometSort - : : +- CometHashAggregate - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometFilter - : : : +- CometSortMergeJoin - : : : :- CometSort - : : : : +- CometExchange - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : : +- CometSubqueryBroadcast - : : : : +- CometBroadcastExchange - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- CometSort - : : : +- CometExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.store_returns - : : +- CometBroadcastExchange - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- CometSort - : +- CometFilter - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometFilter - : : +- CometSortMergeJoin - : : :- CometSort - : : : +- CometExchange - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : +- ReusedSubquery - : : +- CometSort - : : +- CometExchange - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.web_returns - : +- CometBroadcastExchange - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.date_dim - +- CometSort - +- CometFilter - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometFilter - : +- CometSortMergeJoin - : :- CometSort - : : +- CometExchange - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : +- ReusedSubquery - : +- CometSort - : +- CometExchange - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.catalog_returns - +- CometBroadcastExchange - +- CometFilter - +- CometNativeScan parquet spark_catalog.default.date_dim - -Comet accelerated 71 out of 76 eligible operators (93%). Final plan contains 1 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_2/q78.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_2/q78.native_iceberg_compat/extended.txt deleted file mode 100644 index 1b1e6d0cde..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_2/q78.native_iceberg_compat/extended.txt +++ /dev/null @@ -1,80 +0,0 @@ -TakeOrderedAndProject -+- Project [COMET: Comet does not support Spark's BigDecimal rounding] - +- CometNativeColumnarToRow - +- CometSortMergeJoin - :- CometProject - : +- CometSortMergeJoin - : :- CometSort - : : +- CometHashAggregate - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometFilter - : : : +- CometSortMergeJoin - : : : :- CometSort - : : : : +- CometExchange - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometSort - : : : +- CometExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns - : : +- CometBroadcastExchange - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometSort - : +- CometFilter - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometFilter - : : +- CometSortMergeJoin - : : :- CometSort - : : : +- CometExchange - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : +- ReusedSubquery - : : +- CometSort - : : +- CometExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns - : +- CometBroadcastExchange - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - +- CometSort - +- CometFilter - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometFilter - : +- CometSortMergeJoin - : :- CometSort - : : +- CometExchange - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : +- ReusedSubquery - : +- CometSort - : +- CometExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns - +- CometBroadcastExchange - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - -Comet accelerated 70 out of 76 eligible operators (92%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_2/q80a.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_2/q80a.native_datafusion/extended.txt deleted file mode 100644 index 10e0803953..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_2/q80a.native_datafusion/extended.txt +++ /dev/null @@ -1,389 +0,0 @@ -CometNativeColumnarToRow -+- CometTakeOrderedAndProject - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometUnion - :- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometUnion - : :- CometHashAggregate - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometProject - : : : : : : +- CometSortMergeJoin - : : : : : : :- CometSort - : : : : : : : +- CometExchange - : : : : : : : +- CometFilter - : : : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : : : : : +- CometSubqueryBroadcast - : : : : : : : +- CometBroadcastExchange - : : : : : : : +- CometProject - : : : : : : : +- CometFilter - : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : : : +- CometSort - : : : : : : +- CometExchange - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.store_returns - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.store - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.item - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.promotion - : :- CometHashAggregate - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometProject - : : : : : : +- CometSortMergeJoin - : : : : : : :- CometSort - : : : : : : : +- CometExchange - : : : : : : : +- CometFilter - : : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : : : : +- ReusedSubquery - : : : : : : +- CometSort - : : : : : : +- CometExchange - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_returns - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.catalog_page - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.item - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.promotion - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometSortMergeJoin - : : : : : :- CometSort - : : : : : : +- CometExchange - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : : : : +- ReusedSubquery - : : : : : +- CometSort - : : : : : +- CometExchange - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.web_returns - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.web_site - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.item - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.promotion - :- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometUnion - : :- CometHashAggregate - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometProject - : : : : : : +- CometSortMergeJoin - : : : : : : :- CometSort - : : : : : : : +- CometExchange - : : : : : : : +- CometFilter - : : : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : : : : : +- CometSubqueryBroadcast - : : : : : : : +- CometBroadcastExchange - : : : : : : : +- CometProject - : : : : : : : +- CometFilter - : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : : : +- CometSort - : : : : : : +- CometExchange - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.store_returns - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.store - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.item - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.promotion - : :- CometHashAggregate - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometProject - : : : : : : +- CometSortMergeJoin - : : : : : : :- CometSort - : : : : : : : +- CometExchange - : : : : : : : +- CometFilter - : : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : : : : +- ReusedSubquery - : : : : : : +- CometSort - : : : : : : +- CometExchange - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_returns - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.catalog_page - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.item - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.promotion - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometSortMergeJoin - : : : : : :- CometSort - : : : : : : +- CometExchange - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : : : : +- ReusedSubquery - : : : : : +- CometSort - : : : : : +- CometExchange - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.web_returns - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.web_site - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.item - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.promotion - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometUnion - :- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometSortMergeJoin - : : : : : :- CometSort - : : : : : : +- CometExchange - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : : : : +- CometSubqueryBroadcast - : : : : : : +- CometBroadcastExchange - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : : +- CometSort - : : : : : +- CometExchange - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.store_returns - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.store - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.item - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.promotion - :- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometSortMergeJoin - : : : : : :- CometSort - : : : : : : +- CometExchange - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : : : +- ReusedSubquery - : : : : : +- CometSort - : : : : : +- CometExchange - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.catalog_returns - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.catalog_page - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.item - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.promotion - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometSortMergeJoin - : : : : :- CometSort - : : : : : +- CometExchange - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : : : +- ReusedSubquery - : : : : +- CometSort - : : : : +- CometExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.web_returns - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.web_site - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.item - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometNativeScan parquet spark_catalog.default.promotion - -Comet accelerated 377 out of 386 eligible operators (97%). Final plan contains 1 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_2/q80a.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_2/q80a.native_iceberg_compat/extended.txt deleted file mode 100644 index fbd7cba46d..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_2/q80a.native_iceberg_compat/extended.txt +++ /dev/null @@ -1,392 +0,0 @@ -CometNativeColumnarToRow -+- CometTakeOrderedAndProject - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometUnion - :- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometUnion - : :- CometHashAggregate - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometProject - : : : : : : +- CometSortMergeJoin - : : : : : : :- CometSort - : : : : : : : +- CometExchange - : : : : : : : +- 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 - : : : : : : +- 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.date_dim - : : : : +- 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.promotion - : :- CometHashAggregate - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometProject - : : : : : : +- CometSortMergeJoin - : : : : : : :- CometSort - : : : : : : : +- CometExchange - : : : : : : : +- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : : : +- ReusedSubquery - : : : : : : +- CometSort - : : : : : : +- CometExchange - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_page - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.promotion - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometSortMergeJoin - : : : : : :- CometSort - : : : : : : +- CometExchange - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : : : +- ReusedSubquery - : : : : : +- CometSort - : : : : : +- CometExchange - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_site - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.promotion - :- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometUnion - : :- CometHashAggregate - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometProject - : : : : : : +- CometSortMergeJoin - : : : : : : :- CometSort - : : : : : : : +- CometExchange - : : : : : : : +- 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 - : : : : : : +- 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.date_dim - : : : : +- 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.promotion - : :- CometHashAggregate - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometProject - : : : : : : +- CometSortMergeJoin - : : : : : : :- CometSort - : : : : : : : +- CometExchange - : : : : : : : +- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : : : +- ReusedSubquery - : : : : : : +- CometSort - : : : : : : +- CometExchange - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_page - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.promotion - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometSortMergeJoin - : : : : : :- CometSort - : : : : : : +- CometExchange - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : : : +- ReusedSubquery - : : : : : +- CometSort - : : : : : +- CometExchange - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_site - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.promotion - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometUnion - :- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometSortMergeJoin - : : : : : :- CometSort - : : : : : : +- CometExchange - : : : : : : +- 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 - : : : : : +- 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.date_dim - : : : +- 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.promotion - :- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometSortMergeJoin - : : : : : :- CometSort - : : : : : : +- CometExchange - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : : +- ReusedSubquery - : : : : : +- CometSort - : : : : : +- CometExchange - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_page - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.promotion - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometSortMergeJoin - : : : : :- CometSort - : : : : : +- CometExchange - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : : +- ReusedSubquery - : : : : +- CometSort - : : : : +- CometExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_site - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.promotion - -Comet accelerated 374 out of 386 eligible operators (96%). Final plan contains 4 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_2/q86a.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_2/q86a.native_datafusion/extended.txt deleted file mode 100644 index 4c66209a5e..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_2/q86a.native_datafusion/extended.txt +++ /dev/null @@ -1,84 +0,0 @@ -TakeOrderedAndProject -+- 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 - +- CometUnion - :- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : +- CometSubqueryBroadcast - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.item - :- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : +- CometSubqueryBroadcast - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.item - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : +- CometSubqueryBroadcast - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.date_dim - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometNativeScan parquet spark_catalog.default.item - -Comet accelerated 75 out of 81 eligible operators (92%). Final plan contains 1 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_2/q86a.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_2/q86a.native_iceberg_compat/extended.txt deleted file mode 100644 index 58d00a427d..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_2/q86a.native_iceberg_compat/extended.txt +++ /dev/null @@ -1,87 +0,0 @@ -TakeOrderedAndProject -+- 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 - +- CometUnion - :- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- 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 - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - :- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- 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 - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- 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 - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - -Comet accelerated 72 out of 81 eligible operators (88%). Final plan contains 4 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_2/q98.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_2/q98.native_datafusion/extended.txt deleted file mode 100644 index ccf9f25336..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_2/q98.native_datafusion/extended.txt +++ /dev/null @@ -1,32 +0,0 @@ -CometNativeColumnarToRow -+- CometSort - +- CometColumnarExchange - +- Project - +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] - +- CometNativeColumnarToRow - +- CometSort - +- CometExchange - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : +- CometSubqueryBroadcast - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.item - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometNativeScan parquet spark_catalog.default.date_dim - -Comet accelerated 25 out of 28 eligible operators (89%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_2/q98.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_2/q98.native_iceberg_compat/extended.txt deleted file mode 100644 index 1a985ead1b..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_2/q98.native_iceberg_compat/extended.txt +++ /dev/null @@ -1,33 +0,0 @@ -CometNativeColumnarToRow -+- CometSort - +- CometColumnarExchange - +- Project - +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] - +- CometNativeColumnarToRow - +- CometSort - +- CometExchange - +- 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.item - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - -Comet accelerated 24 out of 28 eligible operators (85%). Final plan contains 3 transitions between Spark and Comet. \ No newline at end of file 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 63714383ae..7f353c36e2 100644 --- a/spark/src/test/scala/org/apache/comet/rules/CometExecRuleSuite.scala +++ b/spark/src/test/scala/org/apache/comet/rules/CometExecRuleSuite.scala @@ -31,6 +31,7 @@ import org.apache.spark.sql.execution.exchange.{BroadcastExchangeExec, ShuffleEx import org.apache.spark.sql.types.{DataTypes, StructField, StructType} import org.apache.comet.CometConf +import org.apache.comet.CometSparkSessionExtensions.isSpark40Plus import org.apache.comet.testing.{DataGenOptions, FuzzDataGenerator} /** @@ -131,9 +132,8 @@ class CometExecRuleSuite extends CometTestBase { } } - // TODO this test exposes the bug described in - // https://github.com/apache/datafusion-comet/issues/1389 - ignore("CometExecRule should not allow Comet partial and Spark final hash aggregate") { + // Regression test for https://github.com/apache/datafusion-comet/issues/1389 + test("CometExecRule should not allow Comet partial and Spark final hash aggregate") { withTempView("test_data") { createTestDataFrame.createOrReplaceTempView("test_data") @@ -149,7 +149,8 @@ class CometExecRuleSuite extends CometTestBase { CometConf.COMET_EXEC_LOCAL_TABLE_SCAN_ENABLED.key -> "true") { val transformedPlan = applyCometExecRule(sparkPlan) - // if the final aggregate cannot be converted to Comet, then neither should be + // SUM has incompatible intermediate buffers, so if the final aggregate cannot + // be converted to Comet, neither should be assert( countOperators(transformedPlan, classOf[HashAggregateExec]) == originalHashAggCount) assert(countOperators(transformedPlan, classOf[CometHashAggregateExec]) == 0) @@ -181,6 +182,78 @@ class CometExecRuleSuite extends CometTestBase { } } + test("CometExecRule should allow safe Comet partial and Spark final hash aggregate") { + withTempView("test_data") { + createTestDataFrame.createOrReplaceTempView("test_data") + + // Query uses only safe aggregates (MIN, MAX) with compatible intermediate buffers + val sparkPlan = + createSparkPlan(spark, "SELECT MIN(id), MAX(id) FROM test_data GROUP BY (id % 3)") + + val originalHashAggCount = countOperators(sparkPlan, classOf[HashAggregateExec]) + assert(originalHashAggCount == 2) + + withSQLConf( + CometConf.COMET_ENABLE_FINAL_HASH_AGGREGATE.key -> "false", + CometConf.COMET_EXEC_LOCAL_TABLE_SCAN_ENABLED.key -> "true") { + val transformedPlan = applyCometExecRule(sparkPlan) + + // Safe aggregates allow mixed execution: partial can be Comet, final stays Spark + assert(countOperators(transformedPlan, classOf[HashAggregateExec]) == 1) // final only + assert(countOperators(transformedPlan, classOf[CometHashAggregateExec]) == 1) // partial + } + } + } + + test("CometExecRule should allow safe Spark partial and Comet final hash aggregate") { + withTempView("test_data") { + createTestDataFrame.createOrReplaceTempView("test_data") + + // Query uses only safe aggregates (MIN, MAX) with compatible intermediate buffers + val sparkPlan = + createSparkPlan(spark, "SELECT MIN(id), MAX(id) FROM test_data GROUP BY (id % 3)") + + val originalHashAggCount = countOperators(sparkPlan, classOf[HashAggregateExec]) + assert(originalHashAggCount == 2) + + withSQLConf( + CometConf.COMET_ENABLE_PARTIAL_HASH_AGGREGATE.key -> "false", + CometConf.COMET_EXEC_LOCAL_TABLE_SCAN_ENABLED.key -> "true") { + val transformedPlan = applyCometExecRule(sparkPlan) + + // Safe aggregates allow mixed execution: partial stays Spark, final can be Comet + assert(countOperators(transformedPlan, classOf[HashAggregateExec]) == 1) // partial only + assert(countOperators(transformedPlan, classOf[CometHashAggregateExec]) == 1) // final + } + } + } + + test("CometExecRule should not convert hash aggregate when grouping key contains map type") { + // Spark 3.4/3.5 reject `array>` as a grouping key in the analyzer (not orderable), + // so the plan never reaches CometExecRule on those versions. The guard we're exercising + // (containsMapType) only matters on Spark 4.0+, which permits the GROUP BY to be analyzed. + assume(isSpark40Plus) + // Arrow's row format, used by DataFusion's grouped hash aggregate for composite keys, does + // not support Map at any nesting level. Grouping by a type that transitively contains a map + // (e.g. array>) must stay on Spark to avoid a native row-encoding crash. + val sparkPlan = createSparkPlan( + spark, + """SELECT count(*) + |FROM VALUES (ARRAY(MAP(1, 2), MAP(1, 3))), + | (ARRAY(MAP(2, 3), MAP(1, 3))) AS t(a) + |GROUP BY a""".stripMargin) + + val originalHashAggCount = countOperators(sparkPlan, classOf[HashAggregateExec]) + assert(originalHashAggCount == 2) + + withSQLConf(CometConf.COMET_EXEC_LOCAL_TABLE_SCAN_ENABLED.key -> "true") { + val transformedPlan = applyCometExecRule(sparkPlan) + + assert(countOperators(transformedPlan, classOf[HashAggregateExec]) == originalHashAggCount) + assert(countOperators(transformedPlan, classOf[CometHashAggregateExec]) == 0) + } + } + test("CometExecRule should apply broadcast exchange transformations") { withTempView("test_data") { createTestDataFrame.createOrReplaceTempView("test_data")